You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/28 20:46:36 UTC

[GitHub] [arrow] icexelloss opened a new pull request, #13028: [WIP][ARROW-16083][C++] Implement AsofJoin execution node

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

   ## Overview
   
   This is a work in progress implementation of the AsofJoin node in Arrow C++ compute. The code needs quite a bit of clean up but I have worked on this long enough that I think I benefit from some inputs/comments from Arrow maintainers about the high levels. 
   
   All Credit to @stmrtn (Steven Martin) who is the original author of the code.
   
   ## Implementation
   There are quite a bit of code and here is how it works at the high level:
   
   Classes:
   * `InputState`: A class that handles queuing for input batches and purging unneeded batches. There are one input state per input table.
   * `MemoStore`: A class that responsible for advancing row index and getting the latest row for each key for each key given a timestamp. (Latest timestamp that is <= the given timestamp) 
   * `CompositeReferenceTable`: A class that is responsible for storing temporary output rows and produces RecordBatches from those rows. 
   
   Algorithm:
   * The node takes one left side table and n right side tables, and produces a joined table
   * It is currently assumed that each input table will call `InputReceived` with time-ordered batches. `InputReceived` will queue the batches inside `InputState` (it doesn't do any work). There is a separate process thread that wakes up when there is new inputs and attempts to produces a output batch. If the current data is not enough to produce the output batch (i.e., we have not received all the potential right side rows that could be a match for the current left batch), it will wait for new inputs.
   * The process thread works as follows:
     1. Advance left row index for the current batch. Then advance right tables to get the latest right row (i.e., latest right row with timestamp <= left row timestamp)
     2. Once advances are done, it will continue to check to produce the output row for the current left row
     3. Go to 1 until left batches are processed 
     4. Output batch for the current left batch
     5. Purge batches that are no longer needed
     6. Wait until enough batches are received to process the next left batch
   
   Entry point for the algorithm is `process()`
   
   ## TODO
   - [ ] More Tests
   - [ ] Decide if we can replace `CompositeReferenceTable` with sth that already exits (perhaps `RowEncoder`?)
   - [ ] Handle more datatypes (both key and value)
   - [ ] Handle multiple keys
   - [ ] Life cycle management for the process thread (or whether or not we should have it)
   - [ ] Lint & Code Style
   


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join.h:
##########
@@ -0,0 +1,42 @@
+// 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.
+
+#pragma once
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+
+namespace arrow {
+namespace compute {
+
+typedef int32_t KeyType;

Review Comment:
   Agree. I will move this to the .cc file. This will likely to be removed anyway when we add multiple key support (will add a comment for it).



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

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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column
+  FieldRef time;

Review Comment:
   Renamed to "on_key" and "by_key"



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   I have not removed all debug statement because I expect to make some revisions. I will remove them last. For now please ignore the debug print statements.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column
+  FieldRef time;
+  // keys used for the join. All tables must have the same join key.
+  FieldRef keys;

Review Comment:
   I agree. However, I'd prefer to leave as is since I will like to change this anyway when integrating with substrait (i.e. comments here https://github.com/apache/arrow/pull/13028/files#r882274989)



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column
+  FieldRef time;
+  // keys used for the join. All tables must have the same join key.
+  FieldRef keys;

Review Comment:
   I agree. However, I'd prefer to leave as is since I will like to change this anyway when integrating with substrait (i.e. comments here https://github.com/apache/arrow/pull/13028/files#r882274989) if that is 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] icexelloss commented on a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join.h:
##########
@@ -0,0 +1,42 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   Yes will remove this .h file



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column

Review Comment:
   I added tests for the unsupported types.



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @icexelloss Apologies, I've spent this week (and most of the last) moving.  I'm in my new location now so I should be able to get to this soon (ideally tomorrow).


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>

Review Comment:
   removed api.h



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>

Review Comment:
   remove debug 



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   Gotcha. Is there some suggested way that I can replicate the gcc warning locally? Or should I just submit to CI to check (probably much longer circle time)


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

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

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


[GitHub] [arrow] westonpace commented on a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed

Review Comment:
   We can keep an eye on it.  It sounds like this would only be an issue if the process thread became a bottleneck.



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   Thanks @westonpace that's very helpful. I think I know how to proceed now (pending the CI failure issue).


-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   Thank you for the contribution.  Yes, there would need to be some significant style changes but it would be a welcome addition.
   
   I'll try and give it a more detailed look tomorrow and play around with it.  At a glance a few thoughts:
   
    * We don't want to use any blocking queues, they will need to be resizable.  The execution engine follows a thread-per-core model so any blocking thread is wasted resources (and a potential deadlock).  Instead, when the queue "limit" is surpassed, we should call PauseProducing on the inputs until the queue is sufficiently drained.
    * What happens if the key column(s) have very many values?  I didn't trace all the paths but I think that means the memo store could get quite large and become a memory pressure concern.  Maybe it is only a concern for malicious inputs and we can just reject the query as invalid.  Long term we could probably investigate just storing the row and not the batch, or, if even that is too large, spilling the memo table to disk.  I don't think any of this is something we need to solve now, just random thoughts.
    * We have some utilities for working with row-major data that we had to come up with for the hash-join.  I don't recall if these are in the current implementation, the [proposed implementation](https://github.com/apache/arrow/pull/12326), or both.  However, I bet we can find some overlap here to share utilities.  I'll try and figure out some suggestions after a detailed look.
    * What approach are you thinking wrt sequencing as discussed on the mailing list?
    * Have you given much thought to what an as-of join looks like in Substrait (for example, the current `keys` option won't work I don't think because it is name based (if I understand the meaning) and not index based)
    * Happy to give some thoughts on threading.  My gut instinct is that threading the join itself won't be critically important for most hardware but I could easily be very wrong on 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 #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   @westonpace reading back your comments - I wonder if you can explain a bit more "thread-per-core" model here?
   
   >The execution engine follows a thread-per-core model so any blocking thread is wasted resources (and a potential deadlock)
   The term seems to be used in different context so want to make sure I understand it correctly


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.

Review Comment:
   Reworded



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed

Review Comment:
   Nothing explicitly removes elements from `_ptr2ref`. However, the CompositeReferenceTable object is a temporary object  for each process loop. 
   
   When one of the process loop starts, we create an CompositeReferenceTable and add rows to it, when the process finishes, we output the result batch the the CompositeReferenceTable object will go out of scope, and the unordered_map will be freed.
   
   The reason for this map is that during the process and before materializing the result batch, there is a reference to the input batches in the map so it will not be freed. 



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});

Review Comment:
   sounds good



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>

Review Comment:
   moved



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();

Review Comment:
   Originally I used this to mimic the pattern of HashJoinNode but you are right this is used only once for the schema. I can remove it.



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

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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();

Review Comment:
   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 pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @westonpace I have pushed another revision and addressed most of the comments (and replied the ones that I didn't address or have questions)
   
   Please take another look thank you!


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;

Review Comment:
   Oops, good catch.  This was added in C++17.



-- 
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 #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   Thanks for the explanation!


-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   Thanks @westonpace 
   
   > We don't want to use any blocking queues, they will need to be resizable. The execution engine follows a thread-per-core model so any blocking thread is wasted resources (and a potential deadlock). Instead, when the queue "limit" is surpassed, we should call PauseProducing on the inputs until the queue is sufficiently drained.
   
   I see. I will try to change this.
   
   > What happens if the key column(s) have very many values? I didn't trace all the paths but I think that means the memo store could get quite large and become a memory pressure concern. Maybe it is only a concern for malicious inputs and we can just reject the query as invalid. Long term we could probably investigate just storing the row and not the batch, or, if even that is too large, spilling the memo table to disk. I don't think any of this is something we need to solve now, just random thoughts.
   
   Hmm..not sure if I follow what u mean... Memo store just stores row/index into the batch (it only keeps the latest row for each batch for each key). Even if there are many keys they might just point to the same batch, so it should be a pretty light weight data structure (a couple of pointers per key). Maybe you are concerning many key values pointing to different batches?
   
   >We have some utilities for working with row-major data that we had to come up with for the hash-join. I don't recall if these are in the current implementation, the https://github.com/apache/arrow/pull/12326, or both. However, I bet we can find some overlap here to share utilities. I'll try and figure out some suggestions after a detailed look.
   
   Cool thanks. I was sure if I should reuse `RowEncoder` or not.
   
   > What approach are you thinking wrt sequencing as discussed on the mailing list?
   
   I haven't put too much thought into it and want to leave it out of the scope of this PR. (Seems like adding foundation logic for ordered data could be a separate PR).  At the high level, I think upstream provides batch index and downstream node reorder them and split out to desk is a reasonable approach, just not sure if there is other solution that is simpler.
   
   > Have you given much thought to what an as-of join looks like in Substrait (for example, the current keys option won't work I don't think because it is name based (if I understand the meaning) and not index based)
   
   Yes - @rtpsw did some POC on Substrait + Asof Join and managed to get it to work. We did change to index based from name based in the substrait plan IIRC.
   
   > Happy to give some thoughts on threading. My gut instinct is that threading the join itself won't be critically important for most hardware but I could easily be very wrong on this.
   
   Cool 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] icexelloss commented on pull request #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   @westonpace I am trying to extend the implementation to support multiple keys and key types and wonder if you can give some pointers.
   
   Basically I think I would create a "mapper" that maps an input "row" to the "key" and use that as the the hash map key for the given row. This mapper would 
   * take the column name/index that are key during initialization
   * maps a batch + row index -> key
   * I am not sure what the type of the "key" is but looking at hash join it seems to use just "string" as the key type (using RowEncoder::encoded_row)
   
   I also take a look at aggregation for other options but didn't find anything obvious. 
   
   Did such a "mapper" class already exist in Arrow compute that I can use for this purpose?


-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   Thanks @westonpace I will work on this more Today. Yeah I'd like to get something working and cleaned up first with some baseline performance.


-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   Thanks @westonpace that's super helpful. I will take a look at those classes. Sounds like it's probably worthwhile to take a look at 
   `arrow::compute::Hashing64::HashMultiColumn` if that's what we are moving to for hash join.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   Seems like CI was running
   ```
   + cmake -DARROW_BOOST_USE_SHARED=ON -DARROW_BUILD_BENCHMARKS_REFERENCE=OFF -DARROW_BUILD_BENCHMARKS=OFF -DARROW_BUILD_EXAMPLES=OFF -DARROW_BUILD_INTEGRATION=OFF -DARROW_BUILD_SHARED=ON -DARROW_BUILD_STATIC=OFF -DARROW_BUILD_TESTS=ON -DARROW_BUILD_UTILITIES=ON -DARROW_COMPUTE=ON -DARROW_CSV=ON -DARROW_CUDA=OFF -DARROW_CXXFLAGS= -DARROW_DATASET=ON -DARROW_DEPENDENCY_SOURCE=SYSTEM -DARROW_ENABLE_TIMING_TESTS=OFF -DARROW_EXTRA_ERROR_CONTEXT=OFF -DARROW_FILESYSTEM=ON -DARROW_FLIGHT=OFF -DARROW_FLIGHT_SQL=OFF -DARROW_FUZZING=ON -DARROW_GANDIVA_JAVA=OFF -DARROW_GANDIVA_PC_CXX_FLAGS= -DARROW_GANDIVA=ON -DARROW_GCS=ON -DARROW_HDFS=ON -DARROW_HIVESERVER2=OFF -DARROW_INSTALL_NAME_RPATH=OFF -DARROW_JEMALLOC=OFF -DARROW_JNI=OFF -DARROW_JSON=ON -DARROW_LARGE_MEMORY_TESTS=OFF -DARROW_MIMALLOC=OFF -DARROW_NO_DEPRECATED_API=ON -DARROW_ORC=OFF -DARROW_PARQUET=ON -DARROW_PLASMA_JAVA_CLIENT=OFF -DARROW_PLASMA=ON -DARROW_PYTHON=OFF -DARROW_RUNTIME_SIMD_LEVEL=MAX -DARROW_S3=OFF -DARROW_SKYHOOK=OFF -DAR
 ROW_SUBSTRAIT=ON -DARROW_TEST_LINKAGE=shared -DARROW_TEST_MEMCHECK=OFF -DARROW_USE_ASAN=ON -DARROW_USE_CCACHE=ON -DARROW_USE_GLOG=OFF -DARROW_USE_LD_GOLD=OFF -DARROW_USE_PRECOMPILED_HEADERS=OFF -DARROW_USE_STATIC_CRT=OFF -DARROW_USE_TSAN=OFF -DARROW_USE_UBSAN=ON -DARROW_VERBOSE_THIRDPARTY_BUILD=OFF -DARROW_WITH_BROTLI=ON -DARROW_WITH_BZ2=ON -DARROW_WITH_LZ4=ON -DARROW_WITH_OPENTELEMETRY=ON -DARROW_WITH_SNAPPY=ON -DARROW_WITH_UTF8PROC=ON -DARROW_WITH_ZLIB=ON -DARROW_WITH_ZSTD=ON -DAWSSDK_SOURCE=BUNDLED -Dbenchmark_SOURCE= -DBOOST_SOURCE= -DBrotli_SOURCE= -DBUILD_WARNING_LEVEL=CHECKIN -Dc-ares_SOURCE= -DCMAKE_BUILD_TYPE=debug -DCMAKE_VERBOSE_MAKEFILE=OFF -DCMAKE_C_FLAGS= -DCMAKE_CXX_FLAGS= -DCMAKE_CXX_STANDARD=11 -DCMAKE_INSTALL_LIBDIR=lib -DCMAKE_INSTALL_PREFIX=/usr/local -DCMAKE_UNITY_BUILD=OFF -Dgflags_SOURCE= -Dgoogle_cloud_cpp_storage_SOURCE=BUNDLED -DgRPC_SOURCE=BUNDLED -DGTest_SOURCE=BUNDLED -DLz4_SOURCE= -DORC_SOURCE=BUNDLED -DPARQUET_BUILD_EXAMPLES=ON -DPARQUET_BUILD_EXECUTAB
 LES=ON -DPARQUET_REQUIRE_ENCRYPTION=ON -DProtobuf_SOURCE=BUNDLED -DRapidJSON_SOURCE= -Dre2_SOURCE= -DSnappy_SOURCE= -DThrift_SOURCE= -Dutf8proc_SOURCE=BUNDLED -Dzstd_SOURCE= -G Ninja /arrow/cpp
   ```
   
   But not which option would affect the build time error/warning


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {

Review Comment:
   I agree. If it is ok, I'd like to change this in follow ups for the substrait integration.



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());

Review Comment:
   Added



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());

Review Comment:
   ```suggestion
               return Status::Invalid("Unsupported type for on key: ", field->name());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->type());

Review Comment:
   ```suggestion
               return Status::Invalid("Unsupported type for by key: ", field->name());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->type());
+          }
+          // Only add by field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else {
+          if (supported_data_types_.find(field->type()) == supported_data_types_.end()) {
+            return Status::Invalid("Unsupported data type:", field->type());

Review Comment:
   ```suggestion
               return Status::Invalid("Unsupported data type:", field->name());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->type());
+          }
+          // Only add by field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else {
+          if (supported_data_types_.find(field->type()) == supported_data_types_.end()) {
+            return Status::Invalid("Unsupported data type:", field->type());

Review Comment:
   ```suggestion
               return Status::Invalid("Unsupported data type:", field->name());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->type());

Review Comment:
   ```suggestion
               return Status::Invalid("Unsupported type for by key: ", field->name());
   ```



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @westonpace Sorry for the delay - I took some time off and now got back to this. 
   
   I left a question regarding parallel execution test and wonder if you have some thoughts - appreciate your help.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->type());
+          }
+          // Only add by field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else {
+          if (supported_data_types_.find(field->type()) == supported_data_types_.end()) {
+            return Status::Invalid("Unsupported data type:", field->type());
+          }
+
+          fields.push_back(field);
+        }
+      }
+    }
+    return std::make_shared<arrow::Schema>(fields);

Review Comment:
   I am leaning towards keeping as-is, at least this is how pandas behaves. We can certainly change 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 a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column

Review Comment:
   I agree these restrictions are not ideal. I put "Handle more datatypes (both key and value)" as follow ups only in the interest of time.
   
   > However, we should thoroughly document these limitations as well as test for these when the node is created.
   I agree. I can add tests for behaviors for when unsupported types are passed in - do you have a pointer to some existing tests that test similar behaviors? 



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column

Review Comment:
   I agree these restrictions are not ideal. I put "Handle more datatypes (both key and value)" as follow ups only in the interest of time.
   
   > However, we should thoroughly document these limitations as well as test for these when the node is created.
   
   I agree. I can add tests for behaviors for when unsupported types are passed in - do you have a pointer to some existing tests that test similar behaviors? 



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   I was working on some internal stuff but now back to this.
   
   @westonpace What's a good way to replicating the compiler warning level in the CI build (https://github.com/apache/arrow/runs/6472677191?check_suite_focus=true) locally?
   
   I was doing
   ```
   cmake -DBUILD_WARNING_LEVEL=CHECKIN --preset ninja-debug-basic ..
   ```
   
   But that doesn't seem to output the same build warning as the CI
   


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column
+  FieldRef time;

Review Comment:
   I agree. Will rename to "on" key and "by" key (according to pandas terms https://pandas.pydata.org/pandas-docs/version/0.25.0/reference/api/pandas.merge_asof.html#pandas-merge-asof)



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data

Review Comment:
   Hmm, I think it is no longer needed. We require at least 2 input tables. Will remove.



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the

Review Comment:
   ```suggestion
     // latest_time and latest_ref_row to the value that immediately follows the
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast

Review Comment:
   I think I would probably just do...
   ```
   DCHECK_LE(n_rows, std::numeric_limits<int32_t>::max());
   ```



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>

Review Comment:
   ```suggestion
   #include <iostream>
   ```
   Not sure if there are still debugging statements or not but let's remove this when they are pulled out.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>

Review Comment:
   We try to avoid including the `/api.h` headers as they are "kitchen sink" headers and it slows down compilation (versus having lots of smaller includes).  If I remove it I think all I needed to add to get it to compile is:
   
   ```
   #include "arrow/record_batch.h"
   #include "arrow/array/builder_primitive.h"
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>

Review Comment:
   Group these with `set`/`unordered_map`.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name

Review Comment:
   Let's file a JIRA for this work and include it with the TODO.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;

Review Comment:
   Minor nit: We can probably remove the whitespace between these declarations.



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,29 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+/// \brief Make a node which implements asof join operation
+///
+/// This node takes one left table and (n-1) right tables, and asof joins them
+/// together. Batches produced by each inputs must be ordered by the "on" key.
+/// The batch size that this node produces is decided by the left table.
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef on_key, FieldRef by_key, int64_t tolerance)
+      : on_key(std::move(on_key)), by_key(std::move(by_key)), tolerance(tolerance) {}
+
+  // "on" key for the join. Each input table must be sorted by the "on" key. Inexact
+  // match is used on the "on" key. i.e., a row is considiered match iff
+  // left_on - tolerance <= right_on <= left_on.
+  // Currently, "on" key must be an int64 field
+  FieldRef on_key;
+  // "by" key for the join. All tables must have the "by" key. Equity prediciate
+  // are used on the "by" key.
+  // Currently, "by" key must be an int32 field

Review Comment:
   ```suggestion
     // Currently, the "by" key must be an int32 field
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(

Review Comment:
   Rather than copy this from hash join node tests can we move this method to `src/arrow/compute/exec/test_util`?  It seems generally useful.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty

Review Comment:
   Minor nit: Can you move the trailing comment here to the line above so this declaration fits on one line.



##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});
+
+  AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+
+  ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}})
+                .AddToPlan(plan.get()));
+
+  ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen));
+
+  ASSERT_OK_AND_ASSIGN(auto exp_table,
+                       TableFromExecBatches(exp_batches.schema, exp_batches.batches));
+
+  ASSERT_OK_AND_ASSIGN(auto res_table, TableFromExecBatches(exp_batches.schema, res));
+
+  AssertTablesEqual(*exp_table, *res_table,
+                    /*same_chunk_layout=*/true, /*flatten=*/true);
+}
+
+void DoRunBasicTest(const std::vector<util::string_view>& l_data,
+                    const std::vector<util::string_view>& r0_data,
+                    const std::vector<util::string_view>& r1_data,
+                    const std::vector<util::string_view>& exp_data, int64_t tolerance) {
+  auto l_schema =
+      schema({field("time", int64()), field("key", int32()), field("l_v0", float64())});
+  auto r0_schema =
+      schema({field("time", int64()), field("key", int32()), field("r0_v0", float64())});
+  auto r1_schema =
+      schema({field("time", int64()), field("key", int32()), field("r1_v0", float32())});
+
+  auto exp_schema = schema({
+      field("time", int64()),
+      field("key", int32()),
+      field("l_v0", float64()),
+      field("r0_v0", float64()),
+      field("r1_v0", float32()),
+  });
+
+  // Test three table join
+  BatchesWithSchema l_batches, r0_batches, r1_batches, exp_batches;
+  l_batches = GenerateBatchesFromString(l_schema, l_data);
+  r0_batches = GenerateBatchesFromString(r0_schema, r0_data);
+  r1_batches = GenerateBatchesFromString(r1_schema, r1_data);
+  exp_batches = GenerateBatchesFromString(exp_schema, exp_data);
+  CheckRunOutput(l_batches, r0_batches, r1_batches, exp_batches, "time", "key",
+                 tolerance);
+}
+
+void DoRunInvalidTypeTest(const std::shared_ptr<Schema>& l_schema,
+                          const std::shared_ptr<Schema>& r_schema) {
+  BatchesWithSchema l_batches = GenerateBatchesFromString(l_schema, {R"([])"});
+  BatchesWithSchema r_batches = GenerateBatchesFromString(r_schema, {R"([])"});
+
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options("time", "key", 0);
+  Declaration join{"asofjoin", join_options};
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r_batches.schema, r_batches.gen(false, false)}});
+
+  ASSERT_RAISES(Invalid, join.AddToPlan(plan.get()));

Review Comment:
   Ah, I see, we aren't checking for that.  Can we add those checks?



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {

Review Comment:
   I think that is fine.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);

Review Comment:
   Yes, that seems right.  Not sure what I was thinking originally.
   
   Looking at it with fresh eyes I think there is a minor possibility that `batches_processed_` could be concurrently updated if a single input had multiple empty batches arriving at the same time but it might be easier to change `batches_processed_` to an atomic counter instead of using a mutex.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };

Review Comment:
   For a simple struct there is no need for a leading underscore on the field names (e.g. use `time` and not `_time`)



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,29 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+/// \brief Make a node which implements asof join operation
+///
+/// This node takes one left table and (n-1) right tables, and asof joins them
+/// together. Batches produced by each inputs must be ordered by the "on" key.

Review Comment:
   ```suggestion
   /// together. Batches produced by each input must be ordered by the "on" key.
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->type());
+          }
+          // Only add by field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else {
+          if (supported_data_types_.find(field->type()) == supported_data_types_.end()) {
+            return Status::Invalid("Unsupported data type:", field->type());
+          }
+
+          fields.push_back(field);
+        }
+      }
+    }
+    return std::make_shared<arrow::Schema>(fields);

Review Comment:
   Do we want to reorder things so that the `on_key` and the `by_key` are always the first two fields of the output?  Right now I think that will only be true if they are the first two fields of the first input.
   
   On the other hand, maybe it makes sense to keep the input field order as is.  I think that is what is being done for the regular join node in Substrait.



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,29 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+/// \brief Make a node which implements asof join operation
+///
+/// This node takes one left table and (n-1) right tables, and asof joins them

Review Comment:
   ```suggestion
   ///
   /// Note, this API is experimental and will change in the future
   ///
   /// This node takes one left table and any number of right tables, and asof joins them
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;

Review Comment:
   ```suggestion
     std::unordered_map<KeyType, Entry> entries_;
   ```



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,29 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+/// \brief Make a node which implements asof join operation
+///
+/// This node takes one left table and (n-1) right tables, and asof joins them
+/// together. Batches produced by each inputs must be ordered by the "on" key.
+/// The batch size that this node produces is decided by the left table.

Review Comment:
   ```suggestion
   /// This node will output one row for each row in the left table.
   ```



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,29 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+/// \brief Make a node which implements asof join operation
+///
+/// This node takes one left table and (n-1) right tables, and asof joins them
+/// together. Batches produced by each inputs must be ordered by the "on" key.
+/// The batch size that this node produces is decided by the left table.
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef on_key, FieldRef by_key, int64_t tolerance)
+      : on_key(std::move(on_key)), by_key(std::move(by_key)), tolerance(tolerance) {}
+
+  // "on" key for the join. Each input table must be sorted by the "on" key. Inexact
+  // match is used on the "on" key. i.e., a row is considiered match iff
+  // left_on - tolerance <= right_on <= left_on.
+  // Currently, "on" key must be an int64 field
+  FieldRef on_key;
+  // "by" key for the join. All tables must have the "by" key. Equity prediciate
+  // are used on the "by" key.

Review Comment:
   ```suggestion
     // "by" key for the join. All tables must have the "by" key.  Exact equality
     // is used for the "by" key.
   ```



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,29 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+/// \brief Make a node which implements asof join operation
+///
+/// This node takes one left table and (n-1) right tables, and asof joins them
+/// together. Batches produced by each inputs must be ordered by the "on" key.
+/// The batch size that this node produces is decided by the left table.
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef on_key, FieldRef by_key, int64_t tolerance)
+      : on_key(std::move(on_key)), by_key(std::move(by_key)), tolerance(tolerance) {}
+
+  // "on" key for the join. Each input table must be sorted by the "on" key. Inexact

Review Comment:
   Can you fix all these field docs to match the doxygen style used elsewhere in this file (and other public header files).



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately pass the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->type());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->type());
+          }
+          // Only add by field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else {
+          if (supported_data_types_.find(field->type()) == supported_data_types_.end()) {
+            return Status::Invalid("Unsupported data type:", field->type());
+          }
+
+          fields.push_back(field);
+        }
+      }
+    }
+    return std::make_shared<arrow::Schema>(fields);
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    DCHECK_GE(inputs.size(), 2) << "Must have at least two inputs";
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Schema> output_schema,
+                          MakeOutputSchema(inputs, join_options));
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    state_.at(k)->Push(rb);
+    process_.Push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    {
+      std::lock_guard<std::mutex> guard(gate_);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      state_.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    process_.Push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+  }
+  void StopProducing() override {
+    std::cerr << "StopProducing" << std::endl;
+    finished_.MarkFinished();
+  }
+  arrow::Future<> finished() override { return finished_; }
+
+ private:
+  static const std::set<std::shared_ptr<DataType>> supported_on_types_;
+  static const std::set<std::shared_ptr<DataType>> supported_by_types_;
+  static const std::set<std::shared_ptr<DataType>> supported_data_types_;

Review Comment:
   ```suggestion
     static const std::set<std::shared_ptr<DataType>> kSupportedOnTypes;
     static const std::set<std::shared_ptr<DataType>> kSupportedByTypes;
     static const std::set<std::shared_ptr<DataType>> kSupportedDataTypes;
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});

Review Comment:
   We should have at least some testing with slow / parallel inputs.  I don't think a slow input is capable of reordering data but it might be so we can hold off if that is the case.



##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});
+
+  AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+
+  ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}})
+                .AddToPlan(plan.get()));
+
+  ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen));
+
+  ASSERT_OK_AND_ASSIGN(auto exp_table,
+                       TableFromExecBatches(exp_batches.schema, exp_batches.batches));
+
+  ASSERT_OK_AND_ASSIGN(auto res_table, TableFromExecBatches(exp_batches.schema, res));
+
+  AssertTablesEqual(*exp_table, *res_table,
+                    /*same_chunk_layout=*/true, /*flatten=*/true);
+}
+
+void DoRunBasicTest(const std::vector<util::string_view>& l_data,
+                    const std::vector<util::string_view>& r0_data,
+                    const std::vector<util::string_view>& r1_data,
+                    const std::vector<util::string_view>& exp_data, int64_t tolerance) {
+  auto l_schema =
+      schema({field("time", int64()), field("key", int32()), field("l_v0", float64())});
+  auto r0_schema =
+      schema({field("time", int64()), field("key", int32()), field("r0_v0", float64())});
+  auto r1_schema =
+      schema({field("time", int64()), field("key", int32()), field("r1_v0", float32())});
+
+  auto exp_schema = schema({
+      field("time", int64()),
+      field("key", int32()),
+      field("l_v0", float64()),
+      field("r0_v0", float64()),
+      field("r1_v0", float32()),
+  });
+
+  // Test three table join
+  BatchesWithSchema l_batches, r0_batches, r1_batches, exp_batches;
+  l_batches = GenerateBatchesFromString(l_schema, l_data);
+  r0_batches = GenerateBatchesFromString(r0_schema, r0_data);
+  r1_batches = GenerateBatchesFromString(r1_schema, r1_data);
+  exp_batches = GenerateBatchesFromString(exp_schema, exp_data);
+  CheckRunOutput(l_batches, r0_batches, r1_batches, exp_batches, "time", "key",
+                 tolerance);
+}
+
+void DoRunInvalidTypeTest(const std::shared_ptr<Schema>& l_schema,
+                          const std::shared_ptr<Schema>& r_schema) {
+  BatchesWithSchema l_batches = GenerateBatchesFromString(l_schema, {R"([])"});
+  BatchesWithSchema r_batches = GenerateBatchesFromString(r_schema, {R"([])"});
+
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));

Review Comment:
   ```suggestion
     ExecContext exec_ctx;
     ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(&exec_ctx));
   ```
   Nit: What you have is fine, this is just a touch simpler.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];

Review Comment:
   Hmm...I'm not sure I understand what I was thinking then either :laughing: .  I probably confused myself into thinking this was the table and not an individual row.  I see now I think why it was done this way.  This allows the call to `rows_.reserve(...)` down below to fully allocate both rows and columns all at once.
   
   I'm not fully convinced it has to be this way but I suppose that is a problem that can be handled when this limit is encountered.



##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});
+
+  AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+
+  ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}})
+                .AddToPlan(plan.get()));
+
+  ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen));
+
+  ASSERT_OK_AND_ASSIGN(auto exp_table,
+                       TableFromExecBatches(exp_batches.schema, exp_batches.batches));
+
+  ASSERT_OK_AND_ASSIGN(auto res_table, TableFromExecBatches(exp_batches.schema, res));
+
+  AssertTablesEqual(*exp_table, *res_table,
+                    /*same_chunk_layout=*/true, /*flatten=*/true);
+}
+
+void DoRunBasicTest(const std::vector<util::string_view>& l_data,
+                    const std::vector<util::string_view>& r0_data,
+                    const std::vector<util::string_view>& r1_data,
+                    const std::vector<util::string_view>& exp_data, int64_t tolerance) {
+  auto l_schema =
+      schema({field("time", int64()), field("key", int32()), field("l_v0", float64())});
+  auto r0_schema =
+      schema({field("time", int64()), field("key", int32()), field("r0_v0", float64())});
+  auto r1_schema =
+      schema({field("time", int64()), field("key", int32()), field("r1_v0", float32())});
+
+  auto exp_schema = schema({
+      field("time", int64()),
+      field("key", int32()),
+      field("l_v0", float64()),
+      field("r0_v0", float64()),
+      field("r1_v0", float32()),
+  });
+
+  // Test three table join
+  BatchesWithSchema l_batches, r0_batches, r1_batches, exp_batches;
+  l_batches = GenerateBatchesFromString(l_schema, l_data);
+  r0_batches = GenerateBatchesFromString(r0_schema, r0_data);
+  r1_batches = GenerateBatchesFromString(r1_schema, r1_data);
+  exp_batches = GenerateBatchesFromString(exp_schema, exp_data);
+  CheckRunOutput(l_batches, r0_batches, r1_batches, exp_batches, "time", "key",
+                 tolerance);
+}
+
+void DoRunInvalidTypeTest(const std::shared_ptr<Schema>& l_schema,
+                          const std::shared_ptr<Schema>& r_schema) {
+  BatchesWithSchema l_batches = GenerateBatchesFromString(l_schema, {R"([])"});
+  BatchesWithSchema r_batches = GenerateBatchesFromString(r_schema, {R"([])"});
+
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options("time", "key", 0);
+  Declaration join{"asofjoin", join_options};
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r_batches.schema, r_batches.gen(false, false)}});
+
+  ASSERT_RAISES(Invalid, join.AddToPlan(plan.get()));

Review Comment:
   Wouldn't this generate an error because "there is no `time` or `key` field in the schema" instead of an invalid type error?



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);

Review Comment:
   It shouldn't. Process() is called just by the process thread and `outputs_[0]->InputFinished` is only called inside `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] westonpace commented on a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {

Review Comment:
   The way it's written today is that `StopProducing` is basically a request from the user to abort.  It only needs to be implemented by source nodes and they should stop whatever generation / scanning they are currently doing.



-- 
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] stmrtn commented on pull request #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   Some context: The original implementation of this was a research proof of concept that I was experimenting with.  I'd expect it to be significantly revised or re-written for a production version.  There are parts of the code and some complexity that aren't necessary for the version posted here, and I'd expect changes to make it consistent with other arrow ExecNodes.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   > @westonpace Sorry for the delay - I took some time off and now got back to this.
   
   @icexelloss 
   
   No problem.  I'm actually in the middle of moving so I have somewhat spotty availability this week (and probably next week) anyways.  I've responded to your question.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];

Review Comment:
   Not sure I understand "one allocation per plan".. I think each row would have different refs to entries so there would be many allocations of 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 a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {

Review Comment:
   I agree it's ideal to join here. However, due to the way the code currently it, `StopProducing` is called with in the process thread so it cannot join here. Although I am not sure how  `StopProducing` is supposed to be called. i.e., should the process thread call StopProducing? Or should some other thread responsible for calling `StopProducing`?
   
    



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;

Review Comment:
   Update (slightly different)



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join.cc:
##########
@@ -0,0 +1,59 @@
+// 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 "arrow/compute/exec/asof_join.h"
+#include <iostream>
+
+#include <arrow/api.h>
+#include <arrow/dataset/api.h>
+#include <arrow/dataset/plan.h>
+#include <arrow/filesystem/api.h>
+#include <arrow/io/api.h>
+//#include <arrow/io/util_internal.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/ipc/reader.h>
+#include <arrow/ipc/writer.h>
+#include <arrow/util/async_generator.h>
+#include <arrow/util/checked_cast.h>
+#include <arrow/util/counting_semaphore.h>  // so we don't need to require C++20
+#include <arrow/util/optional.h>
+#include <arrow/util/thread_pool.h>
+#include <algorithm>
+#include <atomic>
+#include <future>
+#include <mutex>
+#include <optional>
+#include <thread>
+
+#include <omp.h>
+
+#include "concurrent_bounded_queue.h"
+
+namespace arrow {
+namespace compute {
+
+class AsofJoinBasicImpl : public AsofJoinImpl {};

Review Comment:
   Removed



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,870 @@
+// 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 <iostream>
+#include <unordered_set>
+
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+#include <arrow/api.h>
+#include <arrow/dataset/api.h>
+#include <arrow/dataset/plan.h>
+#include <arrow/filesystem/api.h>
+#include <arrow/io/api.h>
+//#include <arrow/io/util_internal.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/ipc/reader.h>
+#include <arrow/ipc/writer.h>
+#include <arrow/util/async_generator.h>
+#include <arrow/util/checked_cast.h>
+#include <arrow/util/counting_semaphore.h>  // so we don't need to require C++20
+#include <arrow/util/optional.h>
+#include <arrow/util/thread_pool.h>
+#include <algorithm>
+#include <atomic>
+#include <future>
+#include <mutex>
+#include <optional>
+#include <thread>
+
+#include <omp.h>
+
+#include "concurrent_bounded_queue.h"
+
+namespace arrow {
+namespace compute {
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+  
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : _queue(QUEUE_CAPACITY),
+        _wildcard_key(wildcard_key),
+        _schema(schema),
+        _time_col_index(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        _key_col_index(
+            schema->GetFieldIndex(key_col_name))  // TODO: handle missing field name
+  {                                               /*nothing else*/
+  }
+
+  size_t init_src_to_dst_mapping(size_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst.resize(_schema->num_fields());
+    for (int i = 0; i < _schema->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < _schema->num_fields());
+    return (i == _time_col_index) || (i == _key_col_index);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return _latest_ref_row; }
+
+  bool empty() const {
+    if (_latest_ref_row > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return _queue.empty();
+  }
+
+  int count_batches_processed() const { return _batches_processed; }
+  int count_total_batches() const { return _total_batches; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return _queue.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return _queue.unsync_front()
+        ->column_data(_key_col_index)
+        ->GetValues<KeyType>(1)[_latest_ref_row];
+  }
+  int64_t get_latest_time() const {
+    return _queue.unsync_front()
+        ->column_data(_time_col_index)
+        ->GetValues<int64_t>(1)[_latest_ref_row];
+  }
+
+  bool finished() const { return _batches_processed == _total_batches; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+    
+    bool have_active_batch =
+        (_latest_ref_row > 0 /*short circuit the lock on the queue*/) || !_queue.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++_latest_ref_row >= _queue.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++_batches_processed;
+        _latest_ref_row = 0;
+        have_active_batch &= !_queue.try_pop();
+        if (have_active_batch)
+          assert(_queue.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      if (latest_time <=
+          ts)  // if advance() returns true, then the latest_ts must also be valid
+	// Keep advancing right table until we hit the latest row that has
+	// timestamp <= ts. This is because we only need the latest row for the
+	// match given a left ts.  However, for a futre      
+        _memo.store(get_latest_batch(), _latest_ref_row, latest_time, get_latest_key());
+      else
+        break;  // hit a future timestamp -- done updating for now
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      _queue.push(rb);
+    } else {
+      ++_batches_processed;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = _memo.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (_wildcard_key.has_value()) r = _memo.get_entry_for_key(*_wildcard_key);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    _memo.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return _schema; }
+
+  void set_total_batches(int n) {
+    assert(n >=
+           0);  // not sure why arrow uses a signed int for this, but it should be >=0
+    assert(_total_batches == -1);  // shouldn't be set more than once
+    _total_batches = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  concurrent_bounded_queue<std::shared_ptr<RecordBatch>> _queue;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> _wildcard_key;
+  
+  // Schema associated with the input
+  std::shared_ptr<Schema> _schema;
+  
+  // Total number of batches (only int because InputFinished uses int)
+  int _total_batches = -1;
+  
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int _batches_processed = 0;
+  
+  // Index of the time col
+  col_index_t _time_col_index;
+  
+  // Index of the key col
+  col_index_t _key_col_index;
+
+  // Index of the latest row reference within; if >0 then _queue cannot be empty
+  row_index_t _latest_ref_row =
+    0;  // must be < _queue.front()->num_rows() if _queue is non-empty
+  
+  // Stores latest known values for the various keys
+  MemoStore _memo;
+  
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* _batch;  // can be NULL if there's no value
+    row_index_t _row;
+  };
+  Entry _refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of _rows
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> _rows;
+
+  // Total number of tables in the composite table
+  size_t _n_tables;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+ public:
+  CompositeReferenceTable(size_t n_tables) : _n_tables(n_tables) {
+    assert(_n_tables >= 1);
+    assert(_n_tables <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return _rows.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == _n_tables);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      assert(lhs_latest_batch->num_rows() <=
+             MAX_ROWS_PER_BATCH);  // TODO: better error handling
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = _rows.size() + new_batch_size;
+      if (_rows.capacity() < new_capacity) _rows.reserve(new_capacity);
+      // cerr << "new_batch_size=" << new_batch_size << " old_size=" << _rows.size() << "
+      // new_capacity=" << _rows.capacity() << endl;
+    }
+    _rows.resize(_rows.size() + 1);
+    auto& row = _rows.back();
+    row._refs[0]._batch = lhs_latest_batch.get();
+    row._refs[0]._row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row._refs[i]._batch = entry->_batch.get();
+          row._refs[i]._row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row._refs[i]._batch = NULL;
+      row._refs[i]._row = 0;
+    }
+  }
+
+ private:
+  template <class Builder, class PrimitiveType>
+  std::shared_ptr<Array> materialize_primitive_column(size_t i_table, size_t i_col) {
+    Builder builder;
+    // builder.Resize(_rows.size()); // <-- can't just do this -- need to set the bitmask
+    builder.AppendEmptyValues(_rows.size());
+    for (row_index_t i_row = 0; i_row < _rows.size(); ++i_row) {
+      const auto& ref = _rows[i_row]._refs[i_table];
+      if (ref._batch)
+        builder[i_row] =
+            ref._batch->column_data(i_col)->template GetValues<PrimitiveType>(
+                1)[ref._row];
+      // TODO: set null value if ref._batch is null -- currently we don't due to API
+      // limitations of the builders.
+    }
+    std::shared_ptr<Array> result;
+    if (!builder.Finish(&result).ok()) {
+      std::cerr << "Error when creating Arrow array from builder\n";
+      exit(-1);  // TODO: better error handling
+    }
+    return result;
+  }
+
+ public:
+  // Materializes the current reference table into a target record batch
+  std::shared_ptr<RecordBatch> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == _n_tables);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = _rows.size();
+    if (!n_rows) return nullptr;
+
+    // Count output columns (dbg sanitycheck)
+    {
+      int n_out_cols = 0;
+      for (const auto& s : state) n_out_cols += s->get_schema()->num_fields();
+      n_out_cols -=
+          (state.size() - 1) * 2;  // remove column indices for key and time cols on RHS
+      assert(n_out_cols == output_schema->num_fields());
+    }
+
+    // Instance the types we support
+    std::shared_ptr<arrow::DataType> i32_type = arrow::int32();
+    std::shared_ptr<arrow::DataType> i64_type = arrow::int64();
+    std::shared_ptr<arrow::DataType> f64_type = arrow::float64();
+
+    // Build the arrays column-by-column from our rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < _n_tables; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<size_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+          if (field_type->Equals(i32_type)) {
+            arrays.at(i_dst_col) =
+                materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                           i_src_col);
+          } else if (field_type->Equals(i64_type)) {
+            arrays.at(i_dst_col) =
+                materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                           i_src_col);
+          } else if (field_type->Equals(f64_type)) {
+            arrays.at(i_dst_col) =
+                materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                           i_src_col);
+          } else {
+            std::cerr << "Unsupported data type: " << field_type->name() << "\n";
+            exit(-1);  // TODO: validate elsewhere for better error handling
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    // cerr << "materialize END (ndstrows="<< (r?r->num_rows():-1) <<")\n";
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return _rows.empty(); }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  std::shared_ptr<RecordBatch> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options._tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty())
+      for (size_t i = 1; i < _state.size(); ++i)
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options._tolerance);
+
+    // Emit the batch
+    std::shared_ptr<RecordBatch> r =
+        dst.empty() ? nullptr : dst.materialize(output_schema(), _state);
+    return r;
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      std::shared_ptr<RecordBatch> out_rb = process_inner();
+      if (!out_rb) break;
+      ++_progress_batches_produced;
+      ExecBatch out_b(*out_rb);
+      outputs_[0]->InputReceived(this, std::move(out_b));
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      // cerr << "LHS is finished\n";
+      _total_batches_produced = util::make_optional<int>(_progress_batches_produced);
+      std::cerr << "process() finished " << *_total_batches_produced << "\n";
+      StopProducing();
+      assert(_total_batches_produced.has_value());
+      outputs_[0]->InputFinished(this, *_total_batches_produced);
+    }
+  }
+
+  // Status process_thread(size_t /*thread_index*/, int64_t /*task_id*/) {
+  //   std::cerr << "AsOfJoinNode::process_thread started.\n";
+  //   auto result = _process.try_pop();
+
+  //   if (result == util::nullopt) {
+  //     std::cerr << "AsOfJoinNode::process_thread no inputs.\n";
+  //     return Status::OK();
+  //   } else {
+  //     if (result.value()) {
+  //       std::cerr << "AsOfJoinNode::process_thread process.\n";
+  //       process();
+  //     } else {
+  //       std::cerr << "AsOfJoinNode::process_thread done.\n";
+  //       return Status::OK();
+  //     }
+  //   }
+
+  //   return Status::OK();
+  // }
+
+  // Status process_finished(size_t /*thread_index*/) {
+  //   std::cerr << "AsOfJoinNode::process_finished started.\n";
+  //   return Status::OK();
+  // }
+
+  void process_thread() {
+    std::cerr << "AsOfMergeNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsOfMergeNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr,
+               std::unique_ptr<AsofJoinImpl> impl);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+    ARROW_ASSIGN_OR_RAISE(std::unique_ptr<AsofJoinImpl> impl, AsofJoinImpl::MakeBasic());
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, join_options,
+                                           std::move(output_schema),
+                                           std::move(schema_mgr), std::move(impl));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+      // is_finished=_state.at(k)->finished();
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    // bool use_sync_execution = !(plan_->exec_context()->executor());
+    // std::cerr << "StartScheduling\n";
+    // std::cerr << "use_sync_execution: " << use_sync_execution << std::endl;
+    // RETURN_NOT_OK(
+    //               scheduler_->StartScheduling(0 /*thread index*/,
+    //                                           std::move([this](std::function<Status(size_t)>
+    //                                           func) -> Status {
+    //                                                       return
+    //                                                       this->ScheduleTaskCallback(std::move(func));
+    //                                                     }),
+    //                                           1,
+    //                                           use_sync_execution
+    //                                           )
+    //               );
+    // RETURN_NOT_OK(
+    //               scheduler_->StartTaskGroup(0, task_group_process_, 1)
+    //               );
+    // std::cerr << "StartScheduling done\n";
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {
+    std::cerr << "StopProducing" << std::endl;
+    // if(batch_count_.Cancel()) finished_.MarkFinished();
+    finished_.MarkFinished();
+    for (auto&& input : inputs_) input->StopProducing(this);
+  }
+  Future<> finished() override { return finished_; }
+
+  // Status ScheduleTaskCallback(std::function<Status(size_t)> func) {

Review Comment:
   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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);

Review Comment:
   Hmm..unless there is empty left side batch this shouldn't be false 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] icexelloss commented on a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,

Review Comment:
   Yeah - we can fix this with the substrait integration followup



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data

Review Comment:
   Hmm, I think it is no longer needed. We require at least 2 input tables.



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {

Review Comment:
   Ok.  My understanding here has evolved as I was looking at some of the cleanup in https://github.com/apache/arrow/pull/13143
   
   Eventually I believe `StopProducing` will go away from nodes and be handled by a scheduler.  So I think the right place (long term) for the join is actually in the destructor where you currently have it.



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

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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast

Review Comment:
   I ended up removing the comment about the check here because I think it's minor (since we are not combine batches, we cannot really exceed int64_t).
   
   But I am curious if there is a simple way to do casting check?
   
   ```
   DCHECK_EQ(n_rows, (int64_t) n_rows)
   ```
   
   Gives me a compiler warning



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast

Review Comment:
   I ended up removing the comment about the check here because I think it's minor (since we are not combine batches, we cannot really exceed int64_t).
   
   But I am curious if there is a simple way to do casting check?
   
   ```
   DCHECK_EQ(n_rows, (int64_t)n_rows)
   ```
   
   Gives me a compiler warning



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);

Review Comment:
   We shouldn't need to - `process_` is thread safe queue and already guarded by mutex



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {

Review Comment:
   I think this is for consistency with `GetMemoTimeForKey` other similar methods.



-- 
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 #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   @westonpace I am sort of stuck on this PR and can use some help.
   
   - CI
   The CI appears to be failed due to a GTest issue:
   ```
   Building Substrait from source
   -- Building (vendored) mimalloc from source
   CMake Error at cmake_modules/ThirdpartyToolchain.cmake:239 (find_package):
     Could not find a package configuration file provided by "GTest" (requested
     version 1.10.0) with any of the following names:
       GTestConfig.cmake
       gtest-config.cmake
     Add the installation prefix of "GTest" to CMAKE_PREFIX_PATH or set
     "GTest_DIR" to a directory containing one of the above files.  If "GTest"
     provides a separate development package or SDK, be sure it has been
     installed.
   Call Stack (most recent call first):
     cmake_modules/ThirdpartyToolchain.cmake:1972 (resolve_dependency)
     CMakeLists.txt:552 (include)
   ```
   
   I am not sure what causes this issue or if this is something I did - any suggestions?
   
   - Lint & Code Style
   I have been trying to deal with lint the past day or two and has very limited success. I am able to run `ninja format lint clang-tidy lint_cpp_cli` but it takes a very long time (hours?) and reformatted / gave errors on some files that I do not touch. (I think clang-tidy is the one that is particular problematic). I wonder if I did something wrong. (I installed clang-format 12)
   * The iwyu step doesn't seem to work for me. I followed the instruction and run until `./$IWYU_SH match arrow/compute/exec/` but it doesn't seem to remove any includes. I ran
   ```
   cmake .. --preset ninja-debug-basic -DCMAKE_EXPORT_COMPILE_COMMANDS=ON
   ```
   (Didn't run the exact cmake command from the development doc because I it gives me an missing openSSL error when configuring for some subcomponents)
   then 
   ```
   (arrow-dev) icexelloss@LiUbuntu:~/workspace/arrow/cpp/iwyu$ $IWYU_SH match arrow/compute/exec/asof
   include-what-you-use 0.17 based on Ubuntu clang version 13.0.1-2ubuntu2
   Running IWYU on  /home/icexelloss/workspace/arrow/cpp/src/arrow/compute/exec/asof_join.h /home/icexelloss/workspace/arrow/cpp/src/arrow/compute/exec/asof_join_node.cc /home/icexelloss/workspace/arrow/cpp/src/arrow/compute/exec/asof_join_node_test.cc
   ```
   but nothing seem to have changed


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});

Review Comment:
   Hmm..
   
   I tried to change to parallel test by 
   
   ```
     auto exec_ctx =
       arrow::internal::make_unique<ExecContext>(default_memory_pool(), arrow::internal::GetCpuThreadPool());
   ```
   
   but then I see something weird happening - the node seems to receive data out of order. (Note for k=0, input data is out of order). 
   
   ```
   (k=0) time=([
     2000,
     2000
   ])
   InputFinished find
   InputFinished END
   (k=1) time=([
     1500,
     2000,
     2500
   ])
   InputReceived END
   (k=0) time=([
     0,
     0,
     500,
     1000,
     1500,
     1500
   ])
   ```
   
   My understanding was even with parallel execution, each data source should be single thread, which doesn't seem to be be case. Before I dig too deep, I wonder if my understanding is correct about how `arrow::internal::GetCpuThreadPool()` works?



-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,870 @@
+// 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 <iostream>
+#include <unordered_set>
+
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+#include <arrow/api.h>
+#include <arrow/dataset/api.h>
+#include <arrow/dataset/plan.h>
+#include <arrow/filesystem/api.h>
+#include <arrow/io/api.h>
+//#include <arrow/io/util_internal.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/ipc/reader.h>
+#include <arrow/ipc/writer.h>
+#include <arrow/util/async_generator.h>
+#include <arrow/util/checked_cast.h>
+#include <arrow/util/counting_semaphore.h>  // so we don't need to require C++20
+#include <arrow/util/optional.h>
+#include <arrow/util/thread_pool.h>
+#include <algorithm>
+#include <atomic>
+#include <future>
+#include <mutex>
+#include <optional>
+#include <thread>
+
+#include <omp.h>
+
+#include "concurrent_bounded_queue.h"
+
+namespace arrow {
+namespace compute {
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+  
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : _queue(QUEUE_CAPACITY),
+        _wildcard_key(wildcard_key),
+        _schema(schema),
+        _time_col_index(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        _key_col_index(
+            schema->GetFieldIndex(key_col_name))  // TODO: handle missing field name
+  {                                               /*nothing else*/
+  }
+
+  size_t init_src_to_dst_mapping(size_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst.resize(_schema->num_fields());
+    for (int i = 0; i < _schema->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < _schema->num_fields());
+    return (i == _time_col_index) || (i == _key_col_index);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return _latest_ref_row; }
+
+  bool empty() const {
+    if (_latest_ref_row > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return _queue.empty();
+  }
+
+  int count_batches_processed() const { return _batches_processed; }
+  int count_total_batches() const { return _total_batches; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return _queue.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return _queue.unsync_front()
+        ->column_data(_key_col_index)
+        ->GetValues<KeyType>(1)[_latest_ref_row];
+  }
+  int64_t get_latest_time() const {
+    return _queue.unsync_front()
+        ->column_data(_time_col_index)
+        ->GetValues<int64_t>(1)[_latest_ref_row];
+  }
+
+  bool finished() const { return _batches_processed == _total_batches; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+    
+    bool have_active_batch =
+        (_latest_ref_row > 0 /*short circuit the lock on the queue*/) || !_queue.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++_latest_ref_row >= _queue.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++_batches_processed;
+        _latest_ref_row = 0;
+        have_active_batch &= !_queue.try_pop();
+        if (have_active_batch)
+          assert(_queue.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      if (latest_time <=
+          ts)  // if advance() returns true, then the latest_ts must also be valid
+	// Keep advancing right table until we hit the latest row that has
+	// timestamp <= ts. This is because we only need the latest row for the
+	// match given a left ts.  However, for a futre      
+        _memo.store(get_latest_batch(), _latest_ref_row, latest_time, get_latest_key());
+      else
+        break;  // hit a future timestamp -- done updating for now
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      _queue.push(rb);
+    } else {
+      ++_batches_processed;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = _memo.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (_wildcard_key.has_value()) r = _memo.get_entry_for_key(*_wildcard_key);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    _memo.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return _schema; }
+
+  void set_total_batches(int n) {
+    assert(n >=
+           0);  // not sure why arrow uses a signed int for this, but it should be >=0
+    assert(_total_batches == -1);  // shouldn't be set more than once
+    _total_batches = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  concurrent_bounded_queue<std::shared_ptr<RecordBatch>> _queue;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> _wildcard_key;
+  
+  // Schema associated with the input
+  std::shared_ptr<Schema> _schema;
+  
+  // Total number of batches (only int because InputFinished uses int)
+  int _total_batches = -1;
+  
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int _batches_processed = 0;
+  
+  // Index of the time col
+  col_index_t _time_col_index;
+  
+  // Index of the key col
+  col_index_t _key_col_index;
+
+  // Index of the latest row reference within; if >0 then _queue cannot be empty
+  row_index_t _latest_ref_row =
+    0;  // must be < _queue.front()->num_rows() if _queue is non-empty
+  
+  // Stores latest known values for the various keys
+  MemoStore _memo;
+  
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* _batch;  // can be NULL if there's no value
+    row_index_t _row;
+  };
+  Entry _refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of _rows
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> _rows;
+
+  // Total number of tables in the composite table
+  size_t _n_tables;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+ public:
+  CompositeReferenceTable(size_t n_tables) : _n_tables(n_tables) {
+    assert(_n_tables >= 1);
+    assert(_n_tables <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return _rows.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == _n_tables);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      assert(lhs_latest_batch->num_rows() <=
+             MAX_ROWS_PER_BATCH);  // TODO: better error handling
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = _rows.size() + new_batch_size;
+      if (_rows.capacity() < new_capacity) _rows.reserve(new_capacity);
+      // cerr << "new_batch_size=" << new_batch_size << " old_size=" << _rows.size() << "
+      // new_capacity=" << _rows.capacity() << endl;
+    }
+    _rows.resize(_rows.size() + 1);
+    auto& row = _rows.back();
+    row._refs[0]._batch = lhs_latest_batch.get();
+    row._refs[0]._row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row._refs[i]._batch = entry->_batch.get();
+          row._refs[i]._row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row._refs[i]._batch = NULL;
+      row._refs[i]._row = 0;
+    }
+  }
+
+ private:
+  template <class Builder, class PrimitiveType>
+  std::shared_ptr<Array> materialize_primitive_column(size_t i_table, size_t i_col) {
+    Builder builder;
+    // builder.Resize(_rows.size()); // <-- can't just do this -- need to set the bitmask
+    builder.AppendEmptyValues(_rows.size());
+    for (row_index_t i_row = 0; i_row < _rows.size(); ++i_row) {
+      const auto& ref = _rows[i_row]._refs[i_table];
+      if (ref._batch)
+        builder[i_row] =
+            ref._batch->column_data(i_col)->template GetValues<PrimitiveType>(
+                1)[ref._row];
+      // TODO: set null value if ref._batch is null -- currently we don't due to API
+      // limitations of the builders.
+    }
+    std::shared_ptr<Array> result;
+    if (!builder.Finish(&result).ok()) {
+      std::cerr << "Error when creating Arrow array from builder\n";
+      exit(-1);  // TODO: better error handling
+    }
+    return result;
+  }
+
+ public:
+  // Materializes the current reference table into a target record batch
+  std::shared_ptr<RecordBatch> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == _n_tables);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = _rows.size();
+    if (!n_rows) return nullptr;
+
+    // Count output columns (dbg sanitycheck)
+    {
+      int n_out_cols = 0;
+      for (const auto& s : state) n_out_cols += s->get_schema()->num_fields();
+      n_out_cols -=
+          (state.size() - 1) * 2;  // remove column indices for key and time cols on RHS
+      assert(n_out_cols == output_schema->num_fields());
+    }
+
+    // Instance the types we support
+    std::shared_ptr<arrow::DataType> i32_type = arrow::int32();
+    std::shared_ptr<arrow::DataType> i64_type = arrow::int64();
+    std::shared_ptr<arrow::DataType> f64_type = arrow::float64();
+
+    // Build the arrays column-by-column from our rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < _n_tables; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<size_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+          if (field_type->Equals(i32_type)) {
+            arrays.at(i_dst_col) =
+                materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                           i_src_col);
+          } else if (field_type->Equals(i64_type)) {
+            arrays.at(i_dst_col) =
+                materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                           i_src_col);
+          } else if (field_type->Equals(f64_type)) {
+            arrays.at(i_dst_col) =
+                materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                           i_src_col);
+          } else {
+            std::cerr << "Unsupported data type: " << field_type->name() << "\n";
+            exit(-1);  // TODO: validate elsewhere for better error handling
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    // cerr << "materialize END (ndstrows="<< (r?r->num_rows():-1) <<")\n";
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return _rows.empty(); }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  std::shared_ptr<RecordBatch> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options._tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty())
+      for (size_t i = 1; i < _state.size(); ++i)
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options._tolerance);
+
+    // Emit the batch
+    std::shared_ptr<RecordBatch> r =
+        dst.empty() ? nullptr : dst.materialize(output_schema(), _state);
+    return r;
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      std::shared_ptr<RecordBatch> out_rb = process_inner();
+      if (!out_rb) break;
+      ++_progress_batches_produced;
+      ExecBatch out_b(*out_rb);
+      outputs_[0]->InputReceived(this, std::move(out_b));
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      // cerr << "LHS is finished\n";
+      _total_batches_produced = util::make_optional<int>(_progress_batches_produced);
+      std::cerr << "process() finished " << *_total_batches_produced << "\n";
+      StopProducing();
+      assert(_total_batches_produced.has_value());
+      outputs_[0]->InputFinished(this, *_total_batches_produced);
+    }
+  }
+
+  // Status process_thread(size_t /*thread_index*/, int64_t /*task_id*/) {
+  //   std::cerr << "AsOfJoinNode::process_thread started.\n";
+  //   auto result = _process.try_pop();
+
+  //   if (result == util::nullopt) {
+  //     std::cerr << "AsOfJoinNode::process_thread no inputs.\n";
+  //     return Status::OK();
+  //   } else {
+  //     if (result.value()) {
+  //       std::cerr << "AsOfJoinNode::process_thread process.\n";
+  //       process();
+  //     } else {
+  //       std::cerr << "AsOfJoinNode::process_thread done.\n";
+  //       return Status::OK();
+  //     }
+  //   }
+
+  //   return Status::OK();
+  // }
+
+  // Status process_finished(size_t /*thread_index*/) {
+  //   std::cerr << "AsOfJoinNode::process_finished started.\n";
+  //   return Status::OK();
+  // }
+
+  void process_thread() {
+    std::cerr << "AsOfMergeNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsOfMergeNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr,
+               std::unique_ptr<AsofJoinImpl> impl);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+    ARROW_ASSIGN_OR_RAISE(std::unique_ptr<AsofJoinImpl> impl, AsofJoinImpl::MakeBasic());
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, join_options,
+                                           std::move(output_schema),
+                                           std::move(schema_mgr), std::move(impl));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+      // is_finished=_state.at(k)->finished();
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    // bool use_sync_execution = !(plan_->exec_context()->executor());
+    // std::cerr << "StartScheduling\n";
+    // std::cerr << "use_sync_execution: " << use_sync_execution << std::endl;
+    // RETURN_NOT_OK(
+    //               scheduler_->StartScheduling(0 /*thread index*/,
+    //                                           std::move([this](std::function<Status(size_t)>
+    //                                           func) -> Status {
+    //                                                       return
+    //                                                       this->ScheduleTaskCallback(std::move(func));
+    //                                                     }),
+    //                                           1,
+    //                                           use_sync_execution
+    //                                           )
+    //               );
+    // RETURN_NOT_OK(
+    //               scheduler_->StartTaskGroup(0, task_group_process_, 1)
+    //               );
+    // std::cerr << "StartScheduling done\n";
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {
+    std::cerr << "StopProducing" << std::endl;
+    // if(batch_count_.Cancel()) finished_.MarkFinished();
+    finished_.MarkFinished();
+    for (auto&& input : inputs_) input->StopProducing(this);
+  }
+  Future<> finished() override { return finished_; }
+
+  // Status ScheduleTaskCallback(std::function<Status(size_t)> func) {

Review Comment:
   Ignore this: Failed attempt trying to use TaskScheduler to manage the process thread



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately follows the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {

Review Comment:
   ```suggestion
             if (kSupportedOnTypes.find(field->type()) == kSupportedOnTypes.end()) {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately follows the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->name());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->name());
+          }
+          // Only add by field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else {
+          if (supported_data_types_.find(field->type()) == supported_data_types_.end()) {

Review Comment:
   ```suggestion
             if (kSupportedDataTypes.find(field->type()) == kSupportedDataTypes.end()) {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately follows the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->name());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {

Review Comment:
   ```suggestion
             if (kSupportedByTypes.find(field->type()) == kSupportedByTypes.end()) {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t InitSrcToDstMapping(col_index_t dst_offset, bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && IsTimeOrKeyColumn(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& MapSrcToDst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool IsTimeOrKeyColumn(col_index_t i) const {
+    DCHECK_LT(i, schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t GetLatestRow() const { return latest_ref_row_; }
+
+  bool Empty() const {
+    // cannot be empty if ref row is >0 -- can avoid slow queue lock
+    // below
+    if (latest_ref_row_ > 0) return false;
+    return queue_.Empty();
+  }
+
+  int total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& GetLatestBatch() const {
+    return queue_.UnsyncFront();
+  }
+
+  KeyType GetLatestKey() const {
+    return queue_.UnsyncFront()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+
+  int64_t GetLatestTime() const {
+    return queue_.UnsyncFront()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool Finished() const { return batches_processed_ == total_batches_; }
+
+  bool Advance() {
+    // Try advancing to the next row and update latest_ref_row_
+    // Returns true if able to advance, false if not.
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.Empty();
+
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.UnsyncFront()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.TryPop();
+        if (have_active_batch)
+          DCHECK_GT(queue_.UnsyncFront()->num_rows(), 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified timestamp, update
+  // latest_time and latest_ref_row to the value that immediately follows the
+  // specified timestamp.
+  // Returns true if updates were made, false if not.
+  bool AdvanceAndMemoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (Empty()) return false;  // can't advance if empty
+    auto latest_time = GetLatestTime();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = GetLatestTime();
+      // if Advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.Store(GetLatestBatch(), latest_ref_row_, latest_time, GetLatestKey());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (Advance());
+    return updated;
+  }
+
+  void Push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.Push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> GetMemoEntryForKey(KeyType key) {
+    return memo_.GetEntryForKey(key);
+  }
+
+  util::optional<int64_t> GetMemoTimeForKey(KeyType key) {
+    auto r = GetMemoEntryForKey(key);
+    if (r.has_value()) {
+      return (*r)->_time;
+    } else {
+      return util::nullopt;
+    }
+  }
+
+  void RemoveMemoEntriesWithLesserTime(int64_t ts) {
+    memo_.RemoveEntriesWithLesserTime(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    DCHECK_GE(n, 0);
+    DCHECK_EQ(total_batches_, -1) << "Set total batch more than once";
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    DCHECK_GE(n_tables_, 1);
+    DCHECK_LE(n_tables_, MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void Emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    DCHECK_EQ(in.size(), n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->GetLatestKey();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    DCHECK(!in[0]->Empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch = in[0]->GetLatestBatch();
+    row_index_t lhs_latest_row = in[0]->GetLatestRow();
+    int64_t lhs_latest_time = in[0]->GetLatestTime();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    AddRecordBatchRef(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry = in[i]->GetMemoEntryForKey(key);
+      if (opt_entry.has_value()) {
+        DCHECK(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          AddRecordBatchRef(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> Materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    DCHECK_EQ(state.size(), n_tables_);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->MapSrcToDst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          DCHECK(src_field->type()->Equals(dst_field->type()));
+          DCHECK_EQ(src_field->name(), dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int32Builder, int32_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::Int64Builder, int64_t>(i_table,
+                                                                          i_src_col)));
+          } else if (field_type->Equals(arrow::float32())) {
+            ARROW_ASSIGN_OR_RAISE(arrays.at(i_dst_col),
+                                  (MaterializePrimitiveColumn<arrow::FloatBuilder, float>(
+                                      i_table, i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (MaterializePrimitiveColumn<arrow::DoubleBuilder, double>(i_table,
+                                                                          i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "Requires size_t >= 8 bytes";
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void AddRecordBatchRef(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> MaterializePrimitiveColumn(size_t i_table,
+                                                            col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool UpdateRhs() {
+    auto& lhs = *state_.at(0);
+    auto lhs_latest_time = lhs.GetLatestTime();
+    bool any_updated = false;
+    for (size_t i = 1; i < state_.size(); ++i)
+      any_updated |= state_[i]->AdvanceAndMemoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool IsUpToDateWithLhsRow() const {
+    auto& lhs = *state_[0];
+    if (lhs.Empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.GetLatestTime();
+    for (size_t i = 1; i < state_.size(); ++i) {
+      auto& rhs = *state_[i];
+      if (!rhs.Finished()) {
+        // If RHS is finished, then we know it's up to date
+        if (rhs.Empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.GetLatestTime())
+          return false;  // RHS isn't up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> ProcessInner() {
+    DCHECK(!state_.empty());
+    auto& lhs = *state_.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(state_.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.Finished() || lhs.Empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_rhs_advanced = UpdateRhs();
+
+      // If we have received enough inputs to produce the next output batch
+      // (decided by IsUpToDateWithLhsRow), we will perform the join and
+      // materialize the output batch. The join is done by advancing through
+      // the LHS and adding joined row to rows_ (done by Emplace). Finally,
+      // input batches that are no longer needed are removed to free up memory.
+      if (IsUpToDateWithLhsRow()) {
+        dst.Emplace(state_, options_.tolerance);
+        if (!lhs.Advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if (!any_rhs_advanced) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.Empty()) {
+      for (size_t i = 1; i < state_.size(); ++i) {
+        state_[i]->RemoveMemoEntriesWithLesserTime(lhs.GetLatestTime() -
+                                                   options_.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.Materialize(output_schema(), state_);
+    }
+  }
+
+  void Process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(gate_);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = ProcessInner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++batches_produced_;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (state_.at(0)->Finished()) {
+      StopProducing();
+      outputs_[0]->InputFinished(this, batches_produced_);
+    }
+  }
+
+  void ProcessThread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!process_.Pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      Process();
+    }
+  }
+
+  static void ProcessThreadWrapper(AsofJoinNode* node) { node->ProcessThread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema);
+
+  virtual ~AsofJoinNode() {
+    process_.Push(false);  // poison pill
+    process_thread_.join();
+  }
+
+  static arrow::Result<std::shared_ptr<Schema>> MakeOutputSchema(
+      const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+    std::vector<std::shared_ptr<arrow::Field>> fields;
+
+    // Take all non-key, non-time RHS fields
+    for (size_t j = 0; j < inputs.size(); ++j) {
+      const auto& input_schema = inputs[j]->output_schema();
+      for (int i = 0; i < input_schema->num_fields(); ++i) {
+        const auto field = input_schema->field(i);
+        if (field->name() == *options.on_key.name()) {
+          if (supported_on_types_.find(field->type()) == supported_on_types_.end()) {
+            return Status::Invalid("Unsupported type for on key: ", field->name());
+          }
+          // Only add on field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else if (field->name() == *options.by_key.name()) {
+          if (supported_by_types_.find(field->type()) == supported_by_types_.end()) {
+            return Status::Invalid("Unsupported type for by key: ", field->name());
+          }
+          // Only add by field from the left table
+          if (j == 0) {
+            fields.push_back(field);
+          }
+        } else {
+          if (supported_data_types_.find(field->type()) == supported_data_types_.end()) {
+            return Status::Invalid("Unsupported data type:", field->name());
+          }
+
+          fields.push_back(field);
+        }
+      }
+    }
+    return std::make_shared<arrow::Schema>(fields);
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    DCHECK_GE(inputs.size(), 2) << "Must have at least two inputs";
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Schema> output_schema,
+                          MakeOutputSchema(inputs, join_options));
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    state_.at(k)->Push(rb);
+    process_.Push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    {
+      std::lock_guard<std::mutex> guard(gate_);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      state_.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    process_.Push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+  }
+  void StopProducing() override {
+    std::cerr << "StopProducing" << std::endl;
+    finished_.MarkFinished();
+  }
+  arrow::Future<> finished() override { return finished_; }
+
+ private:
+  static const std::set<std::shared_ptr<DataType>> kSupportedOnTypes;
+  static const std::set<std::shared_ptr<DataType>> kSupportedByTypes;
+  static const std::set<std::shared_ptr<DataType>> kSupportedDataTypes;
+
+  arrow::Future<> finished_;
+  // InputStates
+  // Each input state correponds to an input table
+  std::vector<std::unique_ptr<InputState>> state_;
+  std::mutex gate_;
+  AsofJoinNodeOptions options_;
+
+  // Queue for triggering processing of a given input
+  // (a false value is a poison pill)
+  ConcurrentQueue<bool> process_;
+  // Worker thread
+  std::thread process_thread_;
+
+  // In-progress batches produced
+  int batches_produced_ = 0;
+};
+
+AsofJoinNode::AsofJoinNode(ExecPlan* plan, NodeVector inputs,
+                           std::vector<std::string> input_labels,
+                           const AsofJoinNodeOptions& join_options,
+                           std::shared_ptr<Schema> output_schema)
+    : ExecNode(plan, inputs, input_labels,
+               /*output_schema=*/std::move(output_schema),
+               /*num_outputs=*/1),
+      options_(join_options),
+      process_(),
+      process_thread_(&AsofJoinNode::ProcessThreadWrapper, this) {
+  for (size_t i = 0; i < inputs.size(); ++i)
+    state_.push_back(::arrow::internal::make_unique<InputState>(
+        inputs[i]->output_schema(), *options_.on_key.name(), *options_.by_key.name()));
+  col_index_t dst_offset = 0;
+  for (auto& state : state_)
+    dst_offset = state->InitSrcToDstMapping(dst_offset, !!dst_offset);
+
+  finished_ = arrow::Future<>::MakeFinished();
+}
+
+// Currently supported types
+const std::set<std::shared_ptr<DataType>> AsofJoinNode::supported_on_types_ = {int64()};
+const std::set<std::shared_ptr<DataType>> AsofJoinNode::supported_by_types_ = {int32()};
+const std::set<std::shared_ptr<DataType>> AsofJoinNode::supported_data_types_ = {

Review Comment:
   ```suggestion
   const std::set<std::shared_ptr<DataType>> AsofJoinNode::kSupportedOnTypes = {int64()};
   const std::set<std::shared_ptr<DataType>> AsofJoinNode::kSupportedByTypes = {int32()};
   const std::set<std::shared_ptr<DataType>> AsofJoinNode::kSupportedDataTypes = {
   ```



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});
+
+  AsyncGenerator<util::optional<ExecBatch>> sink_gen;
+
+  ASSERT_OK(Declaration::Sequence({join, {"sink", SinkNodeOptions{&sink_gen}}})
+                .AddToPlan(plan.get()));
+
+  ASSERT_FINISHES_OK_AND_ASSIGN(auto res, StartAndCollect(plan.get(), sink_gen));
+
+  ASSERT_OK_AND_ASSIGN(auto exp_table,
+                       TableFromExecBatches(exp_batches.schema, exp_batches.batches));
+
+  ASSERT_OK_AND_ASSIGN(auto res_table, TableFromExecBatches(exp_batches.schema, res));
+
+  AssertTablesEqual(*exp_table, *res_table,
+                    /*same_chunk_layout=*/true, /*flatten=*/true);
+}
+
+void DoRunBasicTest(const std::vector<util::string_view>& l_data,
+                    const std::vector<util::string_view>& r0_data,
+                    const std::vector<util::string_view>& r1_data,
+                    const std::vector<util::string_view>& exp_data, int64_t tolerance) {
+  auto l_schema =
+      schema({field("time", int64()), field("key", int32()), field("l_v0", float64())});
+  auto r0_schema =
+      schema({field("time", int64()), field("key", int32()), field("r0_v0", float64())});
+  auto r1_schema =
+      schema({field("time", int64()), field("key", int32()), field("r1_v0", float32())});
+
+  auto exp_schema = schema({
+      field("time", int64()),
+      field("key", int32()),
+      field("l_v0", float64()),
+      field("r0_v0", float64()),
+      field("r1_v0", float32()),
+  });
+
+  // Test three table join
+  BatchesWithSchema l_batches, r0_batches, r1_batches, exp_batches;
+  l_batches = GenerateBatchesFromString(l_schema, l_data);
+  r0_batches = GenerateBatchesFromString(r0_schema, r0_data);
+  r1_batches = GenerateBatchesFromString(r1_schema, r1_data);
+  exp_batches = GenerateBatchesFromString(exp_schema, exp_data);
+  CheckRunOutput(l_batches, r0_batches, r1_batches, exp_batches, "time", "key",
+                 tolerance);
+}
+
+void DoRunInvalidTypeTest(const std::shared_ptr<Schema>& l_schema,
+                          const std::shared_ptr<Schema>& r_schema) {
+  BatchesWithSchema l_batches = GenerateBatchesFromString(l_schema, {R"([])"});
+  BatchesWithSchema r_batches = GenerateBatchesFromString(r_schema, {R"([])"});
+
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options("time", "key", 0);
+  Declaration join{"asofjoin", join_options};
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r_batches.schema, r_batches.gen(false, false)}});
+
+  ASSERT_RAISES(Invalid, join.AddToPlan(plan.get()));

Review Comment:
   Added test



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,29 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+/// \brief Make a node which implements asof join operation
+///
+/// This node takes one left table and (n-1) right tables, and asof joins them
+/// together. Batches produced by each inputs must be ordered by the "on" key.
+/// The batch size that this node produces is decided by the left table.
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef on_key, FieldRef by_key, int64_t tolerance)
+      : on_key(std::move(on_key)), by_key(std::move(by_key)), tolerance(tolerance) {}
+
+  // "on" key for the join. Each input table must be sorted by the "on" key. Inexact

Review Comment:
   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 pull request #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   Ok, managed to look at it a bit more today.
   
   Your sidecar processing thread is probably fine for a first approach.  Eventually we will probably want to get rid of it with something that looks like:
   
   ```
   InputRecieved(...) {
     // Called for every input
     StoreBatch();
     // Only called sometimes
     if (HaveEnoughToProcess()) {
       Process();
     }
   }
   ```
   
   The main advantage of the above approach is just to avoid scheduling conflicts / context switches that we would have by introducing another busy thread.
   
   Yes, I misunderstood and my concern about running out of memory was not valid.
   
   The `MemoStore` approach seems sound but I bet we could come up with something more efficient later.  Just thinking out loud I would think something like...
   
   ```
    * When a batch arrives immediately compute a column of hashes based on the key columns (this happens as part of InputReceived before any storage or processing.  Doing it in InputReceived will be important when we get to micro batches because you will want to compute the hashes while the data is still in the CPU cache).  Store the column of hashes with the batch.
    * In processing, take your LHS time column and for each RHS table do something like...
   
   MaterializeAsOf(reference_times, rhs_hashes, rhs_times rhs_columns) {
     vector<row_idx_t> rhs_row_indices = Magic(reference_times, rhs_hashes, rhs_times);
     rhs_columns = []
     for (rhs_column in rhs_columns) {
       rhs_columns.append(Take(rhs_column, rhs_row_indices));
     }
     return rhs_columns;
   }
   ```
   
   I'd have to think a bit more on how `Magic` works (and I think it would require an unordered_map) but if you didn't have a key column it would look something like...
   
   ```
   Magic([1, 5, 10], [2, 4, 6, 8, 10]) -> [-1, 1, 4] // I.e. index of latest_rhs_time for each reference_time
   ```
   
   Then `Take` would just be normal take.  But maybe you also have something like...
   
   ```
   Magic([1, 2, 3, 4, 5, 6, 7, 8], [2, 4, 6]) -> [-1, 0, 0, 1, 1, 2, 2, 2]
   ```
   
   I guess `Take` is just a normal `Take` there too.  That would save you from having to write the materialize logic yourself.  This also keeps things "column-based" which will allow for better utilization of SIMD.  Others will probably have much smarter opinions too :smile: so we can always play with performance later.
   
   I could be way off base here too.  I'll leave it to you on how you want to proceed next.  If you want to do more optimization and benchmarking you can.  If you want to get something that works going first then I think we can focus on getting this cleaned up and migrated over to the Arrow style (see https://arrow.apache.org/docs/developers/cpp/development.html#code-style-linting-and-ci ).  We will also want some more tests.  I think my personal preference would be to get something in that is clean with a lot of tests first.  Then get some benchmarks in place.  Then we can play around with performance.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @westonpace 
   
   > -Wshorten-64-to-32 is clang-specific. I'm not sure about -Wunused-result but I wonder if clang vs. gcc is entirely the difference?
   Sorry I am new to C++ development (mostly Python before), can you elaborate what you mean here? (Is my local build using clang and CI build using gcc?)


-- 
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 #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   I took a stab at using `arrow::compute::Hashing64::HashMultiColumn` but it seems lot of what I need is added in this PR:
   https://github.com/apache/arrow/pull/12326
   
   I will probably wait until that is merged use the utility functions added by that PR, e.g.
   
   `Hashing32::HashBatch`


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join.h:
##########
@@ -0,0 +1,42 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   Shall we get rid of this file and move these declarations into the top of `asof_join_node.cc`?  Or, if you are planning on building other utilities around this header file can you move the definition of `AsofJoinSchema::MakeOutputSchema` into an `asof_join.cc`?



##########
cpp/src/arrow/compute/exec/asof_join.h:
##########
@@ -0,0 +1,42 @@
+// 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.
+
+#pragma once
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+
+namespace arrow {
+namespace compute {
+
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64

Review Comment:
   ```suggestion
   constexpr int kMaxJoinTables = 64
   ```
   Macros ignore namespaces and are prone to conflicts.  We try to avoid them when possible.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {

Review Comment:
   ```suggestion
     T Pop() {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {

Review Comment:
   ```suggestion
     util::optional<const Entry*> GetEntryForKey(KeyType key) const {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,

Review Comment:
   Instead of converting from `FieldRef` to `std::string` you should convert from `FieldRef` to ints.  Admittedly, this wouldn't work today because of the shortcut of assuming the key has the same name in each input.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;

Review Comment:
   Can you either provide some more documentation on what this is or remove it until you later make it configurable?



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);

Review Comment:
   Can this ever be false?  Not a problem, just curious.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);

Review Comment:
   ```suggestion
           DCHECK(*opt_entry);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,

Review Comment:
   ```suggestion
     void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
   ```



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column
+  FieldRef time;

Review Comment:
   Is `time` the best name here?  While I agree that this join is typically used for time series analysis I believe the only requirement is that this be an ordered column present in all inputs.  Maybe `asof_key` or something like that?.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {

Review Comment:
   ```suggestion
     void Push(const T& item) {
   ```



##########
cpp/src/arrow/compute/exec/asof_join.h:
##########
@@ -0,0 +1,42 @@
+// 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.
+
+#pragma once
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+
+namespace arrow {
+namespace compute {
+
+typedef int32_t KeyType;

Review Comment:
   a typedef in a header file is ideally avoided or namespaced inside a struct/class of some kind.  This creates `arrow::compute::KeyType` but `KeyType` doesn't make sense across the entire `arrow::compute` namespace.
   
   Admittedly, this is a moot point if we move the contents here into a .cc file.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {

Review Comment:
   ```suggestion
     bool Empty() const {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {

Review Comment:
   ```suggestion
     util::optional<T> TryPop() {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {

Review Comment:
   ```suggestion
     void RemoveEntriesWithLesserTime(int64_t ts) {
   ```
   For brevity I'm going to stop suggesting snake_case -> CamelCase at this point but please fix all the other methods.  We try to only use `snake_case` for simple accessors.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed

Review Comment:
   Hmm...does anything ever remove elements from `_ptr2ref`?  Will that cause a leak hanging on to all batches that contain at least one output row?



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {

Review Comment:
   If I recall correctly, this relies on `time` and `key` having the same name in all of the inputs.  While this works in Acero today it will not work in Substrait (where names are erased from the plan).  Instead, I think we will want to someday move to `time` being a `vector<FieldRef>` and `keys` being (regrettably) a `vector<vector<FieldRef>>` (i.e. N keys across M inputs).



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column
+  FieldRef time;
+  // keys used for the join. All tables must have the same join key.
+  FieldRef keys;

Review Comment:
   ```suggestion
     // key used for the join. All tables must have the same join key.
     FieldRef key;
   ```
   
   If you want to future proof to multiple keys you could make this `std::vector<FieldRef>` and simply reject plans that have more than one key.



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -361,6 +361,19 @@ class ARROW_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
   Expression filter;
 };
 
+class ARROW_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
+ public:
+  AsofJoinNodeOptions(FieldRef time, FieldRef keys, int64_t tolerance)
+      : time(std::move(time)), keys(std::move(keys)), tolerance(tolerance) {}
+
+  // time column

Review Comment:
   There are quite a few restrictions on these options right now.  `time` must be int64?  `key` must be an int64?  The payload columns must meet a pretty restrictive set of types.
   
   I think this is ok as this is an early implementation.  However, we should thoroughly document these limitations as well as test for these when the node is created.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);

Review Comment:
   ```suggestion
       DCHECK_EQ(state.size(), n_tables_);
       DCHECK_GE(state.size(), 1);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }

Review Comment:
   Is this used?



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();

Review Comment:
   This looks like it's only used to create the output schema so I'm not sure why you are storing it as a member variable.  Could this be a static method?  Or, could you just create a local variable here, generate the output schema, and then let it be discarded?



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {
+    std::cerr << "StopProducing" << std::endl;
+    // if(batch_count_.Cancel()) finished_.MarkFinished();
+    finished_.MarkFinished();
+    for (auto&& input : inputs_) input->StopProducing(this);

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);

Review Comment:
   Can this lead to calling `outputs_[0]->InputFinished` twice?  That should only be called once.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());

Review Comment:
   Can you validate there are at least two inputs here?



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;

Review Comment:
   ```suggestion
       return r.has_value() ? (*r)->_time : util::nullopt;
   ```
   Minor nit: elsewhere we just rely on the implicit constructor of `util::optional`



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.

Review Comment:
   Can you reword this description?  I'm not sure I understand `to the latest data prior to that immediate just past`



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {

Review Comment:
   ```suggestion
       if (!empty()) {
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());

Review Comment:
   ```suggestion
       DCHECK_LT(i, schema_->num_fields());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once

Review Comment:
   ```suggestion
       DCHECK_GT(n, 0);
       DCHECK_EQ(total_batches_, -1);  // shouldn't be set more than once
   ```
   Hmm...I'm not entirely sure total batches == 0 is an invalid case (e.g. we might filter out all data with an overly restrictive filter)



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed

Review Comment:
   ```suggestion
           if (have_active_batch)
             DCHECK_GT(queue_.unsync_front()->num_rows(), 0);  // empty batches disallowed
   ```
   
   When we improve the scheduler we can probably remove empty batches entirely.  For now this is fine (I see you filter out empty batches further down)



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);

Review Comment:
   ```suggestion
       DCHECK_EQ(in.size(), n_tables_);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);

Review Comment:
   ```suggestion
       DCHECK_GE(n_tables_, 1);
       DCHECK_LE(n_tables_, MAX_TABLES);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {

Review Comment:
   If you are returning a pointer here why not use `nullptr` as your "no value found" sentinel?  Then you don't need `util::optional`



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];

Review Comment:
   Why not use `std::vector` here and avoid the template parameter?  There should only be one allocation per plan which is pretty minor.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(

Review Comment:
   Someone recently pointed me to `src/arrow/table_builder.h` which may be really useful here.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast

Review Comment:
   Create a follow-up JIRA and then change this comment to `TODO(ARROW-XYZ): `



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());

Review Comment:
   ```suggestion
             DCHECK(src_field->type()->Equals(dst_field->type()));
             DCHECK_EQ(src_field->name(), dst_field->name());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows

Review Comment:
   ```suggestion
       // Build the result
       DCHECK_GE(sizeof(size_t), sizeof(int64_t)) << "AsofJoinNode requires size_t >= 8 bytes";
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;

Review Comment:
   You might be able to do:
   ```
   _ptr2ref.try_emplace((uintptr_t)ref.get(), ref);
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());

Review Comment:
   ```suggestion
       DCHECK(!in[0]->empty());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data

Review Comment:
   Why `_state.size() > 1`?  What does it mean if `_state.size() == 1`?



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {

Review Comment:
   Ideally we would stop the process thread here but it will eventually get stopped when the plan is destroyed so maybe not critical.



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());

Review Comment:
   ```suggestion
         DCHECK(total_batches_produced_.has_value());
   ```



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);

Review Comment:
   Doesn't this need to be guarded with a mutex?



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);

Review Comment:
   Does this need to be stored in a member variable?



##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);
+      StopProducing();
+      assert(total_batches_produced_.has_value());
+      outputs_[0]->InputFinished(this, *total_batches_produced_);
+    }
+  }
+
+  void process_thread() {
+    std::cerr << "AsofJoinNode::process_thread started.\n";
+    for (;;) {
+      if (!_process.pop()) {
+        std::cerr << "AsofJoinNode::process_thread done.\n";
+        return;
+      }
+      process();
+    }
+  }
+
+  static void process_thread_wrapper(AsofJoinNode* node) { node->process_thread(); }
+
+ public:
+  AsofJoinNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
+               const AsofJoinNodeOptions& join_options,
+               std::shared_ptr<Schema> output_schema,
+               std::unique_ptr<AsofJoinSchema> schema_mgr);
+
+  virtual ~AsofJoinNode() {
+    _process.push(false);  // poison pill
+    _process_thread.join();
+  }
+
+  static arrow::Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
+                                       const ExecNodeOptions& options) {
+    std::unique_ptr<AsofJoinSchema> schema_mgr =
+        ::arrow::internal::make_unique<AsofJoinSchema>();
+
+    const auto& join_options = checked_cast<const AsofJoinNodeOptions&>(options);
+    std::shared_ptr<Schema> output_schema =
+        schema_mgr->MakeOutputSchema(inputs, join_options);
+
+    std::vector<std::string> input_labels(inputs.size());
+    input_labels[0] = "left";
+    for (size_t i = 1; i < inputs.size(); ++i) {
+      input_labels[i] = "right_" + std::to_string(i);
+    }
+
+    return plan->EmplaceNode<AsofJoinNode>(plan, inputs, std::move(input_labels),
+                                           join_options, std::move(output_schema),
+                                           std::move(schema_mgr));
+  }
+
+  const char* kind_name() const override { return "AsofJoinNode"; }
+
+  void InputReceived(ExecNode* input, ExecBatch batch) override {
+    // Get the input
+    ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+    size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+    std::cerr << "InputReceived BEGIN (k=" << k << ")\n";
+
+    // Put into the queue
+    auto rb = *batch.ToRecordBatch(input->output_schema());
+
+    _state.at(k)->push(rb);
+    _process.push(true);
+
+    std::cerr << "InputReceived END\n";
+  }
+  void ErrorReceived(ExecNode* input, Status error) override {
+    outputs_[0]->ErrorReceived(this, std::move(error));
+    StopProducing();
+  }
+  void InputFinished(ExecNode* input, int total_batches) override {
+    std::cerr << "InputFinished BEGIN\n";
+    // bool is_finished=false;
+    {
+      std::lock_guard<std::mutex> guard(_gate);
+      std::cerr << "InputFinished find\n";
+      ARROW_DCHECK(std::find(inputs_.begin(), inputs_.end(), input) != inputs_.end());
+      size_t k = std::find(inputs_.begin(), inputs_.end(), input) - inputs_.begin();
+      // cerr << "set_total_batches for input " << k << ": " << total_batches << "\n";
+      _state.at(k)->set_total_batches(total_batches);
+    }
+    // Trigger a process call
+    // The reason for this is that there are cases at the end of a table where we don't
+    // know whether the RHS of the join is up-to-date until we know that the table is
+    // finished.
+    _process.push(true);
+
+    std::cerr << "InputFinished END\n";
+  }
+  Status StartProducing() override {
+    std::cout << "StartProducing"
+              << "\n";
+    finished_ = arrow::Future<>::Make();
+    return Status::OK();
+  }
+  void PauseProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "PauseProducing"
+              << "\n";
+  }
+  void ResumeProducing(ExecNode* output, int32_t counter) override {
+    std::cout << "ResumeProducing"
+              << "\n";
+  }
+  void StopProducing(ExecNode* output) override {
+    DCHECK_EQ(output, outputs_[0]);
+    StopProducing();
+    std::cout << "StopProducing"
+              << "\n";
+  }
+  void StopProducing() override {
+    std::cerr << "StopProducing" << std::endl;
+    // if(batch_count_.Cancel()) finished_.MarkFinished();
+    finished_.MarkFinished();
+    for (auto&& input : inputs_) input->StopProducing(this);
+  }
+  arrow::Future<> finished() override { return finished_; }
+
+ private:
+  std::unique_ptr<AsofJoinSchema> schema_mgr_;
+  arrow::Future<> finished_;
+  // InputStates
+  // Each input state correponds to an input table
+  //
+  std::vector<std::unique_ptr<InputState>> _state;
+  std::mutex _gate;
+  AsofJoinNodeOptions _options;
+
+  // Queue for triggering processing of a given input
+  // (a false value is a poison pill)
+  ConcurrentQueue<bool> _process;
+  // Worker thread
+  std::thread _process_thread;
+
+  // Total batches produced, once we've finished -- only known at completion time.
+  util::optional<int> total_batches_produced_;
+
+  // In-progress batches produced
+  int _progress_batches_produced = 0;
+};
+
+std::shared_ptr<Schema> AsofJoinSchema::MakeOutputSchema(
+    const std::vector<ExecNode*>& inputs, const AsofJoinNodeOptions& options) {
+  std::vector<std::shared_ptr<arrow::Field>> fields;
+  assert(inputs.size() > 1);

Review Comment:
   ```suggestion
     DCHECK_GT(inputs.size(), 1);
   ```



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

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

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


[GitHub] [arrow] westonpace commented on pull request #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   Sure.  "Thread per core" is probably a bit of a misnomer too, but I haven't found a nicer term yet.  The default thread pool size is std::hardware_concurrency which is the maximum number of concurrent threads the hardware supports.  So we do not over-allocate threads.
   
   When dealing with I/O you normally want to make sure the system is doing useful work while the I/O is happening.  One possible solution is the synchronous approach where you create a pool with a lot of threads, more than your CPU can handle.  When I/O is encountered you simply block synchronously on the I/O and let the OS schedule a different thread onto the hardware.
   
   We don't do that today.  Instead we take an asynchronous approach.  To implement this we actually have two thread pools.  The I/O thread pool is sized based on how many concurrent I/O requests make sense (e.g. not very many for HDD and a lot for S3). It is expected these threads are usually in a waiting state.
   
   The second thread pool (the one that, by default, drives the execution engine) is the CPU thread pool.  This thread pool (again, by default) has a fixed size based on the processor hardware.  It's very important not to block a CPU thread because that usually means you are under utilizing the hardware.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(

Review Comment:
   Good call. Moved



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @westonpace This should be good to go. I addressed all your comments and left follow ups. CI is also green.
   
   Let me know you have more comments.
   
   Li


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   `-Wshorten-64-to-32` is clang-specific.  I'm not sure about `-Wunused-result` but I wonder if clang vs. gcc is entirely the difference?


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @westonpace No worries. Appreciate the heads up.


-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   :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] icexelloss commented on a diff in pull request #13028: [WIP][ARROW-16083][C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join.cc:
##########
@@ -0,0 +1,59 @@
+// 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 "arrow/compute/exec/asof_join.h"
+#include <iostream>
+
+#include <arrow/api.h>
+#include <arrow/dataset/api.h>
+#include <arrow/dataset/plan.h>
+#include <arrow/filesystem/api.h>
+#include <arrow/io/api.h>
+//#include <arrow/io/util_internal.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/ipc/reader.h>
+#include <arrow/ipc/writer.h>
+#include <arrow/util/async_generator.h>
+#include <arrow/util/checked_cast.h>
+#include <arrow/util/counting_semaphore.h>  // so we don't need to require C++20
+#include <arrow/util/optional.h>
+#include <arrow/util/thread_pool.h>
+#include <algorithm>
+#include <atomic>
+#include <future>
+#include <mutex>
+#include <optional>
+#include <thread>
+
+#include <omp.h>
+
+#include "concurrent_bounded_queue.h"
+
+namespace arrow {
+namespace compute {
+
+class AsofJoinBasicImpl : public AsofJoinImpl {};

Review Comment:
   This is not really used right now



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node_test.cc:
##########
@@ -0,0 +1,323 @@
+// 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 <gmock/gmock-matchers.h>
+
+#include <numeric>
+#include <random>
+#include <unordered_set>
+
+#include "arrow/api.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/test_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/compute/kernels/row_encoder.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/matchers.h"
+#include "arrow/testing/random.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/util/thread_pool.h"
+
+using testing::UnorderedElementsAreArray;
+
+namespace arrow {
+namespace compute {
+
+BatchesWithSchema GenerateBatchesFromString(
+    const std::shared_ptr<Schema>& schema,
+    const std::vector<util::string_view>& json_strings, int multiplicity = 1) {
+  BatchesWithSchema out_batches{{}, schema};
+
+  std::vector<ValueDescr> descrs;
+  for (auto&& field : schema->fields()) {
+    descrs.emplace_back(field->type());
+  }
+
+  for (auto&& s : json_strings) {
+    out_batches.batches.push_back(ExecBatchFromJSON(descrs, s));
+  }
+
+  size_t batch_count = out_batches.batches.size();
+  for (int repeat = 1; repeat < multiplicity; ++repeat) {
+    for (size_t i = 0; i < batch_count; ++i) {
+      out_batches.batches.push_back(out_batches.batches[i]);
+    }
+  }
+
+  return out_batches;
+}
+
+void CheckRunOutput(const BatchesWithSchema& l_batches,
+                    const BatchesWithSchema& r0_batches,
+                    const BatchesWithSchema& r1_batches,
+                    const BatchesWithSchema& exp_batches, const FieldRef time,
+                    const FieldRef keys, const int64_t tolerance) {
+  auto exec_ctx =
+      arrow::internal::make_unique<ExecContext>(default_memory_pool(), nullptr);
+  ASSERT_OK_AND_ASSIGN(auto plan, ExecPlan::Make(exec_ctx.get()));
+
+  AsofJoinNodeOptions join_options(time, keys, tolerance);
+  Declaration join{"asofjoin", join_options};
+
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{l_batches.schema, l_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r0_batches.schema, r0_batches.gen(false, false)}});
+  join.inputs.emplace_back(Declaration{
+      "source", SourceNodeOptions{r1_batches.schema, r1_batches.gen(false, false)}});

Review Comment:
   My thinking was to leave the executor as `nullptr` but change things like `l_batches.gen(false, false)` to `l_batches.gen(false, true)`.  Each input should still be ordered but since there is some delay there might be some variation in arrival order.  I think right now you will always get (assuming two batches per input)...
   
   ```
   l_batches::0
   l_batches::1
   l_batches::end
   r0_batches::0
   r0_batches::1
   r0_batches::end
   r1_batches::0
   r1_batches::1
   r1_batches::end
   ```
   
   I was hoping it would be possible that adding slow could yield something like...
   
   ```
   l_batches::0
   r0_batches::0
   l_batches::1
   r1_batches::0
   r0_batches::1
   l_batches::end
   r0_batches::end
   r1_batches::1
   r1_batches::end
   ```
   
   However, if that is not what you are seeing, then we don't need to dig too deeply and can worry about it more when we add support for parallelism.



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @westonpace I think this is ready for another look.
   
   I have resolved most CI/Lint issues and there is only one that fails with a seemingly unrelated issue so I'd like to kick off another around of review.


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join.h:
##########
@@ -0,0 +1,42 @@
+// 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.
+
+#pragma once
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/exec_plan.h>
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+
+namespace arrow {
+namespace compute {
+
+typedef int32_t KeyType;

Review Comment:
   deleted .h file and moved to .cc



##########
cpp/src/arrow/compute/exec/asof_join.h:
##########
@@ -0,0 +1,42 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review Comment:
   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] westonpace commented on pull request #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   The second thing you will often see mentioned is the "morsel / batch" model.  When reading data in you often want to read it in largish blocks of data (counter-intuitively, these large blocks are referred to as "morsels").  This can lead to an execution engine that is roughly:
   
   ```
   parallel for morsel in data_source:
     for operator in pipeline:
       morsel = operator(morsel)
     send_to_sink(morsel)
   ```
   
   However, since each operator is often going over the same data, and morsels are often bigger than CPU caches, this can be inefficient.  Instead the ideal approach is:
   
   ```
   for morsel in data_source:
     parallel for l2_batch in morsel:
       for operator in operators:
         l2_batch = operator(l2_batch)
       send_to_sink(l2_batch)
   ```
   
   This is the model we are trying to work towards in the current execution engine (the hash join is pretty close, the projection and filter nodes still have some work to do before they can handle small batches).


-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }

Review Comment:
   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] westonpace commented on a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast

Review Comment:
   I think I would probably just do...
   ```
   DCHECK_LE(n_rows, std::numeric_limits<int32_t>::max());
   ```
   
   ...but I wouldn't be surprised if we have a shortcut for that somewhere that I'm just not aware of yet.



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   @westonpace Gentle ping - Any thing else you would like me to change ?


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

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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,806 @@
+// 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 <iostream>
+#include <set>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+// Remove this when multiple keys and/or types is supported
+typedef int32_t KeyType;
+
+// Maximum number of tables that can be joined
+#define MAX_JOIN_TABLES 64
+typedef uint64_t row_index_t;
+typedef int col_index_t;
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T Pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void Push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> TryPop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool Empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& UnsyncFront() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void Store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> GetEntryForKey(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void RemoveEntriesWithLesserTime(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name)
+      : queue_(),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name

Review Comment:
   I removed this and added logic to handling missing field in `MakeOutputSchema`



-- 
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 #13028: [WIP][ARROW-16083][C++] Implement AsofJoin execution node

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

   <!--
     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] github-actions[bot] commented on pull request #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

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


-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   `arrow::compute::internal::KeyEncoder` converts an array of values into an array of bytes, such that each value is represented by one or more contiguous bytes.  For example, a standard Arrow boolean array is represented by two non-contiguous "bit buffers" of length/8 bytes.  KeyEncoder represents each boolean value with two contiguous bytes, one for validity and one for the value.
   
   `arrow::compute::internal::RowEncoder` combines the representation from multiple `arrow::compute::internal::KeyEncoder` instances into a a `std::string` for the row.  This `std::string` is just a small byte buffer and shouldn't be treated as a "string" in any way.  The bytes are the bytes from each key encoder for that row.
   
   So if you have three key columns then the string will be the bytes for the first column followed by the bytes for the second column followed by the bytes for the third column.  This string can indeed be used as the key for a hash map.
   
   This approach works ok, but is not the most performant.  A newer version is being integrated which uses `arrow::compute::Hashing64::HashMultiColumn` to calculate an array of 8 bytes hashes.  There is no intermediate string that is created.
   
   All of this should not be confused with `arrow::compute::KeyEncoder` which is a different class entirely that is worried about converting from a columnar format to a row-based format.  This is important in hash-join because the output batches are built from random-access into the hash table.  It should not be important for as-of join because the output batches are still built from sequential (though possibly skipping) access to the input tables.


-- 
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 #13028: ARROW-16083]: [WIP][C++] Implement AsofJoin execution node

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

   @westonpace Not sure if you are the best person to review this. I think I got the basics working but I felt it's also a bit messy. So I am looking for any feedback that can help improve it. Especially the around 1. ergonomic around managing thread/execution model and 2. reusing existing utils/classes for certain things. The algorithm itself is not complicated IMO so it's more like "how  do I do it in the Arrow way".


-- 
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 #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   Thank you for asking.  I think I probably should have been clearer.  It sounds like you might be doing more than is needed (we should maybe update our developer docs or put some work into getting these pieces working again).
   
   > The CI appears to be failed due to a GTest issue:
   
   Yes, this particular GTest issue is not related to your change.  I think it's being worked on but you don't need to worry about it.
   
   > Lint & Code Style
   
   I wouldn't worry about running clang-tidy.  As long as you don't get any compiler warnings and pass `clang format` then that should be good enough.  Not all style issues have static checks however.  For example, `concurrent_queue` should be `ConcurrentQueue` but we can do a code review to look for those.
   
   > The iwyu step doesn't seem to work for me.
   
   I don't know that it works for anyone.  I would say that our philosophy is iwyu but we don't have any way of enforcing it.  So this is more of a "if you have any doubts use the rules of iwyu to know what to include" but don't stress too much if you might have missed something.
   
   If you remove `[WIP]` from your title then more CI jobs should run.  As long as these are passing I think we can catch any remaining style issues in code review.


-- 
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 #13028: ARROW-16083: [WIP][C++] Implement AsofJoin execution node

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

   @github-actions autotune


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

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

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


[GitHub] [arrow] icexelloss commented on a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;

Review Comment:
   `try_emplace` looks nice! However, I got the error 
   ```
   error: ‘class std::unordered_map<long unsigned int, std::shared_ptr<arrow::RecordBatch> >’ has no member named ‘try_emplace’; did you mean ‘emplace’?
   ```
   
   Not sure what is going on maybe this is not available in C++ 11? (also try to search for usage of try_emplace in the code base and didn't find any)



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once

Review Comment:
   Total batches == 0 is valid case (added empty test for 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 a diff in pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;

Review Comment:
   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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed

Review Comment:
   Nothing explicitly removes elements from `_ptr2ref`. However, the CompositeReferenceTable object is a temporary object  for each process loop. 
   
   When one of the process loop starts, we create an CompositeReferenceTable and adding rows to it, when the process finishes, we output the result batch the the CompositeReferenceTable object will go out of scope, and the unordered_map will be freed.
   
   The reason for this map is that during the process and before materializing the result batch, there is a reference to the input batches in the map so it will not be freed. 



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;

Review Comment:
   Yeah I can remove this (oversight from the original code)



-- 
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 #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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


##########
cpp/src/arrow/compute/exec/asof_join_node.cc:
##########
@@ -0,0 +1,800 @@
+// 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 <iostream>
+#include <unordered_map>
+
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/util/optional.h>
+#include "arrow/compute/exec/asof_join.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/compute/exec/schema_util.h"
+#include "arrow/compute/exec/util.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/future.h"
+#include "arrow/util/make_unique.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+namespace arrow {
+namespace compute {
+
+/**
+ * Simple implementation for an unbound concurrent queue
+ */
+template <class T>
+class ConcurrentQueue {
+ public:
+  T pop() {
+    std::unique_lock<std::mutex> lock(mutex_);
+    cond_.wait(lock, [&] { return !queue_.empty(); });
+    auto item = queue_.front();
+    queue_.pop();
+    return item;
+  }
+
+  void push(const T& item) {
+    std::unique_lock<std::mutex> lock(mutex_);
+    queue_.push(item);
+    cond_.notify_one();
+  }
+
+  util::optional<T> try_pop() {
+    // Try to pop the oldest value from the queue (or return nullopt if none)
+    std::unique_lock<std::mutex> lock(mutex_);
+    if (queue_.empty()) {
+      return util::nullopt;
+    } else {
+      auto item = queue_.front();
+      queue_.pop();
+      return item;
+    }
+  }
+
+  bool empty() const {
+    std::unique_lock<std::mutex> lock(mutex_);
+    return queue_.empty();
+  }
+
+  // Un-synchronized access to front
+  // For this to be "safe":
+  // 1) the caller logically guarantees that queue is not empty
+  // 2) pop/try_pop cannot be called concurrently with this
+  const T& unsync_front() const { return queue_.front(); }
+
+ private:
+  std::queue<T> queue_;
+  mutable std::mutex mutex_;
+  std::condition_variable cond_;
+};
+
+struct MemoStore {
+  // Stores last known values for all the keys
+
+  struct Entry {
+    // Timestamp associated with the entry
+    int64_t _time;
+
+    // Batch associated with the entry (perf is probably OK for this; batches change
+    // rarely)
+    std::shared_ptr<arrow::RecordBatch> _batch;
+
+    // Row associated with the entry
+    row_index_t _row;
+  };
+
+  std::unordered_map<KeyType, Entry> _entries;
+
+  void store(const std::shared_ptr<RecordBatch>& batch, row_index_t row, int64_t time,
+             KeyType key) {
+    auto& e = _entries[key];
+    // that we can do this assignment optionally, is why we
+    // can get array with using shared_ptr above (the batch
+    // shouldn't change that often)
+    if (e._batch != batch) e._batch = batch;
+    e._row = row;
+    e._time = time;
+  }
+
+  util::optional<const Entry*> get_entry_for_key(KeyType key) const {
+    auto e = _entries.find(key);
+    if (_entries.end() == e) return util::nullopt;
+    return util::optional<const Entry*>(&e->second);
+  }
+
+  void remove_entries_with_lesser_time(int64_t ts) {
+    size_t dbg_size0 = _entries.size();
+    for (auto e = _entries.begin(); e != _entries.end();)
+      if (e->second._time < ts)
+        e = _entries.erase(e);
+      else
+        ++e;
+    size_t dbg_size1 = _entries.size();
+    if (dbg_size1 < dbg_size0) {
+      // cerr << "Removed " << dbg_size0-dbg_size1 << " memo entries.\n";
+    }
+  }
+};
+
+class InputState {
+  // InputState correponds to an input
+  // Input record batches are queued up in InputState until processed and
+  // turned into output record batches.
+
+ public:
+  InputState(const std::shared_ptr<arrow::Schema>& schema,
+             const std::string& time_col_name, const std::string& key_col_name,
+             util::optional<KeyType> wildcard_key)
+      : queue_(),
+        wildcard_key_(wildcard_key),
+        schema_(schema),
+        time_col_index_(
+            schema->GetFieldIndex(time_col_name)),  // TODO: handle missing field name
+        key_col_index_(schema->GetFieldIndex(key_col_name)) {}
+
+  col_index_t init_src_to_dst_mapping(col_index_t dst_offset,
+                                      bool skip_time_and_key_fields) {
+    src_to_dst_.resize(schema_->num_fields());
+    for (int i = 0; i < schema_->num_fields(); ++i)
+      if (!(skip_time_and_key_fields && is_time_or_key_column(i)))
+        src_to_dst_[i] = dst_offset++;
+    return dst_offset;
+  }
+
+  const util::optional<col_index_t>& map_src_to_dst(col_index_t src) const {
+    return src_to_dst_[src];
+  }
+
+  bool is_time_or_key_column(col_index_t i) const {
+    assert(i < schema_->num_fields());
+    return (i == time_col_index_) || (i == key_col_index_);
+  }
+
+  // Gets the latest row index,  assuming the queue isn't empty
+  row_index_t get_latest_row() const { return latest_ref_row_; }
+
+  bool empty() const {
+    if (latest_ref_row_ > 0)
+      return false;  // cannot be empty if ref row is >0 -- can avoid slow queue lock
+                     // below
+    return queue_.empty();
+  }
+
+  int countbatches_processed_() const { return batches_processed_; }
+  int count_total_batches() const { return total_batches_; }
+
+  // Gets latest batch (precondition: must not be empty)
+  const std::shared_ptr<arrow::RecordBatch>& get_latest_batch() const {
+    return queue_.unsync_front();
+  }
+  KeyType get_latest_key() const {
+    return queue_.unsync_front()
+        ->column_data(key_col_index_)
+        ->GetValues<KeyType>(1)[latest_ref_row_];
+  }
+  int64_t get_latest_time() const {
+    return queue_.unsync_front()
+        ->column_data(time_col_index_)
+        ->GetValues<int64_t>(1)[latest_ref_row_];
+  }
+
+  bool finished() const { return batches_processed_ == total_batches_; }
+
+  bool advance() {
+    // Returns true if able to advance, false if not.
+
+    bool have_active_batch =
+        (latest_ref_row_ > 0 /*short circuit the lock on the queue*/) || !queue_.empty();
+    if (have_active_batch) {
+      // If we have an active batch
+      if (++latest_ref_row_ >= (row_index_t)queue_.unsync_front()->num_rows()) {
+        // hit the end of the batch, need to get the next batch if possible.
+        ++batches_processed_;
+        latest_ref_row_ = 0;
+        have_active_batch &= !queue_.try_pop();
+        if (have_active_batch)
+          assert(queue_.unsync_front()->num_rows() > 0);  // empty batches disallowed
+      }
+    }
+    return have_active_batch;
+  }
+
+  // Advance the data to be immediately past the specified TS, updating latest and
+  // latest_ref_row to the latest data prior to that immediate just past Returns true if
+  // updates were made, false if not.
+  bool advance_and_memoize(int64_t ts) {
+    // Advance the right side row index until we reach the latest right row (for each key)
+    // for the given left timestamp.
+
+    // Check if already updated for TS (or if there is no latest)
+    if (empty()) return false;  // can't advance if empty
+    auto latest_time = get_latest_time();
+    if (latest_time > ts) return false;  // already advanced
+
+    // Not updated.  Try to update and possibly advance.
+    bool updated = false;
+    do {
+      latest_time = get_latest_time();
+      // if advance() returns true, then the latest_ts must also be valid
+      // Keep advancing right table until we hit the latest row that has
+      // timestamp <= ts. This is because we only need the latest row for the
+      // match given a left ts.
+      if (latest_time <= ts) {
+        memo_.store(get_latest_batch(), latest_ref_row_, latest_time, get_latest_key());
+      } else {
+        break;  // hit a future timestamp -- done updating for now
+      }
+      updated = true;
+    } while (advance());
+    return updated;
+  }
+
+  void push(const std::shared_ptr<arrow::RecordBatch>& rb) {
+    if (rb->num_rows() > 0) {
+      queue_.push(rb);
+    } else {
+      ++batches_processed_;  // don't enqueue empty batches, just record as processed
+    }
+  }
+
+  util::optional<const MemoStore::Entry*> get_memo_entry_for_key(KeyType key) {
+    auto r = memo_.get_entry_for_key(key);
+    if (r.has_value()) return r;
+    if (wildcard_key_.has_value()) r = memo_.get_entry_for_key(*wildcard_key_);
+    return r;
+  }
+
+  util::optional<int64_t> get_memo_time_for_key(KeyType key) {
+    auto r = get_memo_entry_for_key(key);
+    return r.has_value() ? util::make_optional((*r)->_time) : util::nullopt;
+  }
+
+  void remove_memo_entries_with_lesser_time(int64_t ts) {
+    memo_.remove_entries_with_lesser_time(ts);
+  }
+
+  const std::shared_ptr<Schema>& get_schema() const { return schema_; }
+
+  void set_total_batches(int n) {
+    assert(n >= 0);
+    assert(total_batches_ == -1);  // shouldn't be set more than once
+    total_batches_ = n;
+  }
+
+ private:
+  // Pending record batches.  The latest is the front.  Batches cannot be empty.
+  ConcurrentQueue<std::shared_ptr<RecordBatch>> queue_;
+
+  // Wildcard key for this input, if applicable.
+  util::optional<KeyType> wildcard_key_;
+
+  // Schema associated with the input
+  std::shared_ptr<Schema> schema_;
+
+  // Total number of batches (only int because InputFinished uses int)
+  int total_batches_ = -1;
+
+  // Number of batches processed so far (only int because InputFinished uses int)
+  int batches_processed_ = 0;
+
+  // Index of the time col
+  col_index_t time_col_index_;
+
+  // Index of the key col
+  col_index_t key_col_index_;
+
+  // Index of the latest row reference within; if >0 then queue_ cannot be empty
+  row_index_t latest_ref_row_ =
+      0;  // must be < queue_.front()->num_rows() if queue_ is non-empty
+
+  // Stores latest known values for the various keys
+  MemoStore memo_;
+
+  // Mapping of source columns to destination columns
+  std::vector<util::optional<col_index_t>> src_to_dst_;
+};
+
+template <size_t MAX_TABLES>
+struct CompositeReferenceRow {
+  struct Entry {
+    arrow::RecordBatch* batch;  // can be NULL if there's no value
+    row_index_t row;
+  };
+  Entry refs[MAX_TABLES];
+};
+
+// A table of composite reference rows.  Rows maintain pointers to the
+// constituent record batches, but the overall table retains shared_ptr
+// references to ensure memory remains resident while the table is live.
+//
+// The main reason for this is that, especially for wide tables, joins
+// are effectively row-oriented, rather than column-oriented.  Separating
+// the join part from the columnar materialization part simplifies the
+// logic around data types and increases efficiency.
+//
+// We don't put the shared_ptr's into the rows for efficiency reasons.
+template <size_t MAX_TABLES>
+class CompositeReferenceTable {
+ public:
+  explicit CompositeReferenceTable(size_t n_tables) : n_tables_(n_tables) {
+    assert(n_tables_ >= 1);
+    assert(n_tables_ <= MAX_TABLES);
+  }
+
+  size_t n_rows() const { return rows_.size(); }
+
+  // Adds the latest row from the input state as a new composite reference row
+  // - LHS must have a valid key,timestep,and latest rows
+  // - RHS must have valid data memo'ed for the key
+  void emplace(std::vector<std::unique_ptr<InputState>>& in, int64_t tolerance) {
+    assert(in.size() == n_tables_);
+
+    // Get the LHS key
+    KeyType key = in[0]->get_latest_key();
+
+    // Add row and setup LHS
+    // (the LHS state comes just from the latest row of the LHS table)
+    assert(!in[0]->empty());
+    const std::shared_ptr<arrow::RecordBatch>& lhs_latest_batch =
+        in[0]->get_latest_batch();
+    row_index_t lhs_latest_row = in[0]->get_latest_row();
+    int64_t lhs_latest_time = in[0]->get_latest_time();
+    if (0 == lhs_latest_row) {
+      // On the first row of the batch, we resize the destination.
+      // The destination size is dictated by the size of the LHS batch.
+      row_index_t new_batch_size = lhs_latest_batch->num_rows();
+      row_index_t new_capacity = rows_.size() + new_batch_size;
+      if (rows_.capacity() < new_capacity) rows_.reserve(new_capacity);
+    }
+    rows_.resize(rows_.size() + 1);
+    auto& row = rows_.back();
+    row.refs[0].batch = lhs_latest_batch.get();
+    row.refs[0].row = lhs_latest_row;
+    add_record_batch_ref(lhs_latest_batch);
+
+    // Get the state for that key from all on the RHS -- assumes it's up to date
+    // (the RHS state comes from the memoized row references)
+    for (size_t i = 1; i < in.size(); ++i) {
+      util::optional<const MemoStore::Entry*> opt_entry =
+          in[i]->get_memo_entry_for_key(key);
+      if (opt_entry.has_value()) {
+        assert(*opt_entry);
+        if ((*opt_entry)->_time + tolerance >= lhs_latest_time) {
+          // Have a valid entry
+          const MemoStore::Entry* entry = *opt_entry;
+          row.refs[i].batch = entry->_batch.get();
+          row.refs[i].row = entry->_row;
+          add_record_batch_ref(entry->_batch);
+          continue;
+        }
+      }
+      row.refs[i].batch = NULL;
+      row.refs[i].row = 0;
+    }
+  }
+
+  // Materializes the current reference table into a target record batch
+  Result<std::shared_ptr<RecordBatch>> materialize(
+      const std::shared_ptr<arrow::Schema>& output_schema,
+      const std::vector<std::unique_ptr<InputState>>& state) {
+    // cerr << "materialize BEGIN\n";
+    assert(state.size() == n_tables_);
+    assert(state.size() >= 1);
+
+    // Don't build empty batches
+    size_t n_rows = rows_.size();
+    if (!n_rows) return NULLPTR;
+
+    // Build the arrays column-by-column from the rows
+    std::vector<std::shared_ptr<arrow::Array>> arrays(output_schema->num_fields());
+    for (size_t i_table = 0; i_table < n_tables_; ++i_table) {
+      int n_src_cols = state.at(i_table)->get_schema()->num_fields();
+      {
+        for (col_index_t i_src_col = 0; i_src_col < n_src_cols; ++i_src_col) {
+          util::optional<col_index_t> i_dst_col_opt =
+              state[i_table]->map_src_to_dst(i_src_col);
+          if (!i_dst_col_opt) continue;
+          col_index_t i_dst_col = *i_dst_col_opt;
+          const auto& src_field = state[i_table]->get_schema()->field(i_src_col);
+          const auto& dst_field = output_schema->field(i_dst_col);
+          assert(src_field->type()->Equals(dst_field->type()));
+          assert(src_field->name() == dst_field->name());
+          const auto& field_type = src_field->type();
+
+          if (field_type->Equals(arrow::int32())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int32Builder, int32_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::int64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::Int64Builder, int64_t>(i_table,
+                                                                            i_src_col)));
+          } else if (field_type->Equals(arrow::float64())) {
+            ARROW_ASSIGN_OR_RAISE(
+                arrays.at(i_dst_col),
+                (materialize_primitive_column<arrow::DoubleBuilder, double>(i_table,
+                                                                            i_src_col)));
+          } else {
+            ARROW_RETURN_NOT_OK(
+                Status::Invalid("Unsupported data type: ", src_field->name()));
+          }
+        }
+      }
+    }
+
+    // Build the result
+    assert(sizeof(size_t) >= sizeof(int64_t));  // Make takes signed int64_t for num_rows
+
+    // TODO: check n_rows for cast
+    std::shared_ptr<arrow::RecordBatch> r =
+        arrow::RecordBatch::Make(output_schema, (int64_t)n_rows, arrays);
+    return r;
+  }
+
+  // Returns true if there are no rows
+  bool empty() const { return rows_.empty(); }
+
+ private:
+  // Contains shared_ptr refs for all RecordBatches referred to by the contents of rows_
+  std::unordered_map<uintptr_t, std::shared_ptr<RecordBatch>> _ptr2ref;
+
+  // Row table references
+  std::vector<CompositeReferenceRow<MAX_TABLES>> rows_;
+
+  // Total number of tables in the composite table
+  size_t n_tables_;
+
+  // Adds a RecordBatch ref to the mapping, if needed
+  void add_record_batch_ref(const std::shared_ptr<RecordBatch>& ref) {
+    if (!_ptr2ref.count((uintptr_t)ref.get())) _ptr2ref[(uintptr_t)ref.get()] = ref;
+  }
+
+  template <class Builder, class PrimitiveType>
+  Result<std::shared_ptr<Array>> materialize_primitive_column(size_t i_table,
+                                                              col_index_t i_col) {
+    Builder builder;
+    ARROW_RETURN_NOT_OK(builder.Reserve(rows_.size()));
+    for (row_index_t i_row = 0; i_row < rows_.size(); ++i_row) {
+      const auto& ref = rows_[i_row].refs[i_table];
+      if (ref.batch) {
+        builder.UnsafeAppend(
+            ref.batch->column_data(i_col)->template GetValues<PrimitiveType>(1)[ref.row]);
+      } else {
+        builder.UnsafeAppendNull();
+      }
+    }
+    std::shared_ptr<Array> result;
+    ARROW_RETURN_NOT_OK(builder.Finish(&result));
+    return result;
+  }
+};
+
+class AsofJoinNode : public ExecNode {
+  // Constructs labels for inputs
+  static std::vector<std::string> build_input_labels(
+      const std::vector<ExecNode*>& inputs) {
+    std::vector<std::string> r(inputs.size());
+    for (size_t i = 0; i < r.size(); ++i) r[i] = "input_" + std::to_string(i) + "_label";
+    return r;
+  }
+
+  // Advances the RHS as far as possible to be up to date for the current LHS timestamp
+  bool update_rhs() {
+    auto& lhs = *_state.at(0);
+    auto lhs_latest_time = lhs.get_latest_time();
+    bool any_updated = false;
+    for (size_t i = 1; i < _state.size(); ++i)
+      any_updated |= _state[i]->advance_and_memoize(lhs_latest_time);
+    return any_updated;
+  }
+
+  // Returns false if RHS not up to date for LHS
+  bool is_up_to_date_for_lhs_row() const {
+    auto& lhs = *_state[0];
+    if (lhs.empty()) return false;  // can't proceed if nothing on the LHS
+    int64_t lhs_ts = lhs.get_latest_time();
+    for (size_t i = 1; i < _state.size(); ++i) {
+      auto& rhs = *_state[i];
+      if (!rhs.finished()) {
+        // If RHS is finished, then we know it's up to date (but if it isn't, it might be
+        // up to date)
+        if (rhs.empty())
+          return false;  // RHS isn't finished, but is empty --> not up to date
+        if (lhs_ts >= rhs.get_latest_time())
+          return false;  // TS not up to date (and not finished)
+      }
+    }
+    return true;
+  }
+
+  Result<std::shared_ptr<RecordBatch>> process_inner() {
+    assert(!_state.empty());
+    auto& lhs = *_state.at(0);
+
+    // Construct new target table if needed
+    CompositeReferenceTable<MAX_JOIN_TABLES> dst(_state.size());
+
+    // Generate rows into the dst table until we either run out of data or hit the row
+    // limit, or run out of input
+    for (;;) {
+      // If LHS is finished or empty then there's nothing we can do here
+      if (lhs.finished() || lhs.empty()) break;
+
+      // Advance each of the RHS as far as possible to be up to date for the LHS timestamp
+      bool any_advanced = update_rhs();
+
+      // Only update if we have up-to-date information for the LHS row
+      if (is_up_to_date_for_lhs_row()) {
+        dst.emplace(_state, _options.tolerance);
+        if (!lhs.advance()) break;  // if we can't advance LHS, we're done for this batch
+      } else {
+        if ((!any_advanced) && (_state.size() > 1)) break;  // need to wait for new data
+      }
+    }
+
+    // Prune memo entries that have expired (to bound memory consumption)
+    if (!lhs.empty()) {
+      for (size_t i = 1; i < _state.size(); ++i) {
+        _state[i]->remove_memo_entries_with_lesser_time(lhs.get_latest_time() -
+                                                        _options.tolerance);
+      }
+    }
+
+    // Emit the batch
+    if (dst.empty()) {
+      return NULLPTR;
+    } else {
+      return dst.materialize(output_schema(), _state);
+    }
+  }
+
+  void process() {
+    std::cerr << "process() begin\n";
+
+    std::lock_guard<std::mutex> guard(_gate);
+    if (finished_.is_finished()) {
+      std::cerr << "InputReceived EARLYEND\n";
+      return;
+    }
+
+    // Process batches while we have data
+    for (;;) {
+      Result<std::shared_ptr<RecordBatch>> result = process_inner();
+
+      if (result.ok()) {
+        auto out_rb = *result;
+        if (!out_rb) break;
+        ++_progress_batches_produced;
+        ExecBatch out_b(*out_rb);
+        outputs_[0]->InputReceived(this, std::move(out_b));
+      } else {
+        StopProducing();
+        ErrorIfNotOk(result.status());
+        return;
+      }
+    }
+
+    std::cerr << "process() end\n";
+
+    // Report to the output the total batch count, if we've already finished everything
+    // (there are two places where this can happen: here and InputFinished)
+    //
+    // It may happen here in cases where InputFinished was called before we were finished
+    // producing results (so we didn't know the output size at that time)
+    if (_state.at(0)->finished()) {
+      total_batches_produced_ = util::make_optional<int>(_progress_batches_produced);

Review Comment:
   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] westonpace commented on pull request #13028: ARROW-16083: [C++] Implement AsofJoin execution node

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

   > Sorry I am new to C++ development (mostly Python before), can you elaborate what you mean here? (Is my local build using clang and CI build using gcc?)
   
   @icexelloss 
   
   Yes, but the other way around.  There are a number of different C/C++ compilers (our CI runs on, at least, gcc, clang, msvc, and mingw).  Both gcc and clang are popular on Linux systems and gcc is probably more common at this point.  Clang is used somewhat exclusively in macos environments and has been gaining in popularity on Linux recently too.
   
   Each compiler has its own quirks and each compiler supports a different set of warnings and some even interpret a particular warning in different ways.  `-Wshorten-64-to-32` is only supported on clang (msvc will check for this as well but it has a different name for it).  Gcc has `-Wconversion` but that covers a lot more things (some of which we tolerate in the Arrow code base) and so we disable that warning on gcc.  So, if you are compiling locally with gcc, then you will never see those warnings until you submit to CI.
   
   Ultimately, code will need to pass without warnings on all of the compilers (except for the warnings we specifically tolerate) before we merge it in.  I don't know of a great way to do this.  My own work tends to involve a certain amount of "lint/warning" commits which I fail to catch before it gets to CI so I can understand the annoyance.


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