Initial commit
This commit is contained in:
@@ -0,0 +1,162 @@
|
||||
// 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 <cstdint>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/compute/exec.h"
|
||||
#include "arrow/result.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
namespace util {
|
||||
|
||||
using arrow::compute::ExecBatch;
|
||||
|
||||
/// \brief A container that accumulates batches until they are ready to
|
||||
/// be processed.
|
||||
class ARROW_ACERO_EXPORT AccumulationQueue {
|
||||
public:
|
||||
AccumulationQueue() : row_count_(0) {}
|
||||
~AccumulationQueue() = default;
|
||||
|
||||
// We should never be copying ExecBatch around
|
||||
AccumulationQueue(const AccumulationQueue&) = delete;
|
||||
AccumulationQueue& operator=(const AccumulationQueue&) = delete;
|
||||
|
||||
AccumulationQueue(AccumulationQueue&& that);
|
||||
AccumulationQueue& operator=(AccumulationQueue&& that);
|
||||
|
||||
void Concatenate(AccumulationQueue&& that);
|
||||
void InsertBatch(ExecBatch batch);
|
||||
int64_t row_count() { return row_count_; }
|
||||
size_t batch_count() { return batches_.size(); }
|
||||
bool empty() const { return batches_.empty(); }
|
||||
void Clear();
|
||||
ExecBatch& operator[](size_t i);
|
||||
|
||||
private:
|
||||
int64_t row_count_;
|
||||
std::vector<ExecBatch> batches_;
|
||||
};
|
||||
|
||||
/// A queue that sequences incoming batches
|
||||
///
|
||||
/// This can be used when a node needs to do some kind of ordered processing on
|
||||
/// the stream.
|
||||
///
|
||||
/// Batches can be inserted in any order. The process_callback will be called on
|
||||
/// the batches, in order, without reentrant calls. For this reason the callback
|
||||
/// should be quick.
|
||||
///
|
||||
/// For example, in a top-n node, the process callback should determine how many
|
||||
/// rows need to be delivered for the given batch, and then return a task to actually
|
||||
/// deliver those rows.
|
||||
class ARROW_ACERO_EXPORT SequencingQueue {
|
||||
public:
|
||||
using Task = std::function<Status()>;
|
||||
|
||||
/// Strategy that describes how to handle items
|
||||
class Processor {
|
||||
public:
|
||||
/// Process the batch, potentially generating a task
|
||||
///
|
||||
/// This method will be called on each batch in order. Calls to this method
|
||||
/// will be serialized and it will not be called reentrantly. This makes it
|
||||
/// safe to do things that rely on order but minimal time should be spent here
|
||||
/// to avoid becoming a bottleneck.
|
||||
///
|
||||
/// \return a follow-up task that will be scheduled. The follow-up task(s) are
|
||||
/// is not guaranteed to run in any particular order. If nullopt is
|
||||
/// returned then nothing will be scheduled.
|
||||
virtual Result<std::optional<Task>> Process(ExecBatch batch) = 0;
|
||||
/// Schedule a task
|
||||
virtual void Schedule(Task task) = 0;
|
||||
};
|
||||
|
||||
virtual ~SequencingQueue() = default;
|
||||
|
||||
/// Insert a batch into the queue
|
||||
///
|
||||
/// This will insert the batch into the queue. If this batch was the next batch
|
||||
/// to deliver then this will trigger 1+ calls to the process callback to generate
|
||||
/// 1+ tasks.
|
||||
///
|
||||
/// The task generated by this call will be executed immediately. The remaining
|
||||
/// tasks will be scheduled using the schedule callback.
|
||||
///
|
||||
/// From a data pipeline perspective the sequencing queue is a "sometimes" breaker. If
|
||||
/// a task arrives in order then this call will usually execute the downstream pipeline.
|
||||
/// If this task arrives early then this call will only queue the data.
|
||||
virtual Status InsertBatch(ExecBatch batch) = 0;
|
||||
|
||||
/// Create a queue
|
||||
/// \param processor describes how to process the batches, must outlive the queue
|
||||
static std::unique_ptr<SequencingQueue> Make(Processor* processor);
|
||||
};
|
||||
|
||||
/// A queue that sequences incoming batches
|
||||
///
|
||||
/// Unlike SequencingQueue the Process method is not expected to schedule new tasks.
|
||||
///
|
||||
/// If a batch arrives and another thread is currently processing then the batch
|
||||
/// will be queued and control will return. In other words, delivery of batches will
|
||||
/// not block on the Process method.
|
||||
///
|
||||
/// It can be helpful to think of this as if a dedicated thread is running Process as
|
||||
/// batches arrive
|
||||
class ARROW_ACERO_EXPORT SerialSequencingQueue {
|
||||
public:
|
||||
/// Strategy that describes how to handle items
|
||||
class Processor {
|
||||
public:
|
||||
virtual ~Processor() = default;
|
||||
/// Process the batch
|
||||
///
|
||||
/// This method will be called on each batch in order. Calls to this method
|
||||
/// will be serialized and it will not be called reentrantly. This makes it
|
||||
/// safe to do things that rely on order.
|
||||
///
|
||||
/// If this falls behind then data may accumulate
|
||||
///
|
||||
/// TODO: Could add backpressure if needed but right now all uses of this should
|
||||
/// be pretty fast and so are unlikely to block.
|
||||
virtual Status Process(ExecBatch batch) = 0;
|
||||
};
|
||||
|
||||
virtual ~SerialSequencingQueue() = default;
|
||||
|
||||
/// Insert a batch into the queue
|
||||
///
|
||||
/// This will insert the batch into the queue. If this batch was the next batch
|
||||
/// to deliver then this may trigger calls to the processor which will be run
|
||||
/// as part of this call.
|
||||
virtual Status InsertBatch(ExecBatch batch) = 0;
|
||||
|
||||
/// Create a queue
|
||||
/// \param processor describes how to process the batches, must outlive the queue
|
||||
static std::unique_ptr<SerialSequencingQueue> Make(Processor* processor);
|
||||
};
|
||||
|
||||
} // namespace util
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,58 @@
|
||||
// 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.
|
||||
|
||||
// This API is EXPERIMENTAL.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/compute/api_aggregate.h"
|
||||
#include "arrow/compute/test_util_internal.h"
|
||||
#include "arrow/compute/type_fwd.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/type_fwd.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
namespace aggregate {
|
||||
|
||||
using compute::Aggregate;
|
||||
using compute::default_exec_context;
|
||||
using compute::ExecContext;
|
||||
|
||||
/// \brief Make the output schema of an aggregate node
|
||||
///
|
||||
/// The output schema is determined by the aggregation kernels, which may depend on the
|
||||
/// ExecContext argument. To guarantee correct results, the same ExecContext argument
|
||||
/// should be used in execution.
|
||||
///
|
||||
/// \param[in] input_schema the schema of the input to the node
|
||||
/// \param[in] keys the grouping keys for the aggregation
|
||||
/// \param[in] segment_keys the segmenting keys for the aggregation
|
||||
/// \param[in] aggregates the aggregates for the aggregation
|
||||
/// \param[in] exec_ctx the execution context for the aggregation
|
||||
ARROW_ACERO_EXPORT Result<std::shared_ptr<Schema>> MakeOutputSchema(
|
||||
const std::shared_ptr<Schema>& input_schema, const std::vector<FieldRef>& keys,
|
||||
const std::vector<FieldRef>& segment_keys, const std::vector<Aggregate>& aggregates,
|
||||
ExecContext* exec_ctx = default_exec_context());
|
||||
|
||||
} // namespace aggregate
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,32 @@
|
||||
// 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.
|
||||
|
||||
// NOTE: API is EXPERIMENTAL and will change without going through a
|
||||
// deprecation cycle
|
||||
|
||||
#pragma once
|
||||
|
||||
/// \defgroup acero-api Utilities for creating and executing execution plans
|
||||
/// @{
|
||||
/// @}
|
||||
|
||||
/// \defgroup acero-nodes Options classes for the various exec nodes
|
||||
/// @{
|
||||
/// @}
|
||||
|
||||
#include "arrow/acero/exec_plan.h"
|
||||
#include "arrow/acero/options.h"
|
||||
@@ -0,0 +1,41 @@
|
||||
// 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 <vector>
|
||||
|
||||
#include "arrow/acero/options.h"
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/compute/exec.h"
|
||||
#include "arrow/type.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
namespace asofjoin {
|
||||
|
||||
using AsofJoinKeys = AsofJoinNodeOptions::Keys;
|
||||
|
||||
/// \brief Make the output schema of an as-of-join node
|
||||
///
|
||||
/// \param[in] input_schema the schema of each input to the node
|
||||
/// \param[in] input_keys the key of each input to the node
|
||||
ARROW_ACERO_EXPORT Result<std::shared_ptr<Schema>> MakeOutputSchema(
|
||||
const std::vector<std::shared_ptr<Schema>>& input_schema,
|
||||
const std::vector<AsofJoinKeys>& input_keys);
|
||||
|
||||
} // namespace asofjoin
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,64 @@
|
||||
// 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/acero/exec_plan.h"
|
||||
#include "arrow/acero/options.h"
|
||||
|
||||
#include <memory>
|
||||
|
||||
namespace arrow::acero {
|
||||
|
||||
class BackpressureHandler {
|
||||
private:
|
||||
BackpressureHandler(size_t low_threshold, size_t high_threshold,
|
||||
std::unique_ptr<BackpressureControl> backpressure_control)
|
||||
: low_threshold_(low_threshold),
|
||||
high_threshold_(high_threshold),
|
||||
backpressure_control_(std::move(backpressure_control)) {}
|
||||
|
||||
public:
|
||||
static Result<BackpressureHandler> Make(
|
||||
size_t low_threshold, size_t high_threshold,
|
||||
std::unique_ptr<BackpressureControl> backpressure_control) {
|
||||
if (low_threshold >= high_threshold) {
|
||||
return Status::Invalid("low threshold (", low_threshold,
|
||||
") must be less than high threshold (", high_threshold, ")");
|
||||
}
|
||||
if (backpressure_control == NULLPTR) {
|
||||
return Status::Invalid("null backpressure control parameter");
|
||||
}
|
||||
BackpressureHandler backpressure_handler(low_threshold, high_threshold,
|
||||
std::move(backpressure_control));
|
||||
return backpressure_handler;
|
||||
}
|
||||
|
||||
void Handle(size_t start_level, size_t end_level) {
|
||||
if (start_level < high_threshold_ && end_level >= high_threshold_) {
|
||||
backpressure_control_->Pause();
|
||||
} else if (start_level > low_threshold_ && end_level <= low_threshold_) {
|
||||
backpressure_control_->Resume();
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
size_t low_threshold_;
|
||||
size_t high_threshold_;
|
||||
std::unique_ptr<BackpressureControl> backpressure_control_;
|
||||
};
|
||||
|
||||
} // namespace arrow::acero
|
||||
@@ -0,0 +1,48 @@
|
||||
// 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 <cstdint>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "benchmark/benchmark.h"
|
||||
|
||||
#include "arrow/acero/exec_plan.h"
|
||||
#include "arrow/acero/test_util_internal.h"
|
||||
#include "arrow/compute/exec.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
namespace acero {
|
||||
|
||||
Status BenchmarkNodeOverhead(benchmark::State& state, int32_t num_batches,
|
||||
int32_t batch_size, arrow::acero::BatchesWithSchema data,
|
||||
std::vector<arrow::acero::Declaration>& node_declarations,
|
||||
arrow::MemoryPool* pool = default_memory_pool());
|
||||
|
||||
Status BenchmarkIsolatedNodeOverhead(benchmark::State& state,
|
||||
arrow::compute::Expression expr, int32_t num_batches,
|
||||
int32_t batch_size,
|
||||
arrow::acero::BatchesWithSchema data,
|
||||
std::string factory_name,
|
||||
arrow::acero::ExecNodeOptions& options,
|
||||
arrow::MemoryPool* pool = default_memory_pool());
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -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.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
#include "arrow/acero/partition_util.h"
|
||||
#include "arrow/acero/util.h"
|
||||
#include "arrow/memory_pool.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/status.h"
|
||||
#include "arrow/util/simd.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
|
||||
// A set of pre-generated bit masks from a 64-bit word.
|
||||
//
|
||||
// It is used to map selected bits of hash to a bit mask that will be used in
|
||||
// a Bloom filter.
|
||||
//
|
||||
// These bit masks need to look random and need to have a similar fractions of
|
||||
// bits set in order for a Bloom filter to have a low false positives rate.
|
||||
//
|
||||
struct ARROW_ACERO_EXPORT BloomFilterMasks {
|
||||
// Generate all masks as a single bit vector. Each bit offset in this bit
|
||||
// vector corresponds to a single mask.
|
||||
// In each consecutive kBitsPerMask bits, there must be between
|
||||
// kMinBitsSet and kMaxBitsSet bits set.
|
||||
//
|
||||
BloomFilterMasks();
|
||||
|
||||
inline uint64_t mask(int bit_offset) {
|
||||
#if ARROW_LITTLE_ENDIAN
|
||||
return (arrow::util::SafeLoadAs<uint64_t>(masks_ + bit_offset / 8) >>
|
||||
(bit_offset % 8)) &
|
||||
kFullMask;
|
||||
#else
|
||||
return (BYTESWAP(arrow::util::SafeLoadAs<uint64_t>(masks_ + bit_offset / 8)) >>
|
||||
(bit_offset % 8)) &
|
||||
kFullMask;
|
||||
#endif
|
||||
}
|
||||
|
||||
// Masks are 57 bits long because then they can be accessed at an
|
||||
// arbitrary bit offset using a single unaligned 64-bit load instruction.
|
||||
//
|
||||
static constexpr int kBitsPerMask = 57;
|
||||
static constexpr uint64_t kFullMask = (1ULL << kBitsPerMask) - 1;
|
||||
|
||||
// Minimum and maximum number of bits set in each mask.
|
||||
// This constraint is enforced when generating the bit masks.
|
||||
// Values should be close to each other and chosen as to minimize a Bloom
|
||||
// filter false positives rate.
|
||||
//
|
||||
static constexpr int kMinBitsSet = 4;
|
||||
static constexpr int kMaxBitsSet = 5;
|
||||
|
||||
// Number of generated masks.
|
||||
// Having more masks to choose will improve false positives rate of Bloom
|
||||
// filter but will also use more memory, which may lead to more CPU cache
|
||||
// misses.
|
||||
// The chosen value results in using only a few cache-lines for mask lookups,
|
||||
// while providing a good variety of available bit masks.
|
||||
//
|
||||
static constexpr int kLogNumMasks = 10;
|
||||
static constexpr int kNumMasks = 1 << kLogNumMasks;
|
||||
|
||||
// Data of masks. Masks are stored in a single bit vector. Nth mask is
|
||||
// kBitsPerMask bits starting at bit offset N.
|
||||
//
|
||||
static constexpr int kTotalBytes = (kNumMasks + 64) / 8;
|
||||
uint8_t masks_[kTotalBytes];
|
||||
};
|
||||
|
||||
// A variant of a blocked Bloom filter implementation.
|
||||
// A Bloom filter is a data structure that provides approximate membership test
|
||||
// functionality based only on the hash of the key. Membership test may return
|
||||
// false positives but not false negatives. Approximation of the result allows
|
||||
// in general case (for arbitrary data types of keys) to save on both memory and
|
||||
// lookup cost compared to the accurate membership test.
|
||||
// The accurate test may sometimes still be cheaper for a specific data types
|
||||
// and inputs, e.g. integers from a small range.
|
||||
//
|
||||
// This blocked Bloom filter is optimized for use in hash joins, to achieve a
|
||||
// good balance between the size of the filter, the cost of its building and
|
||||
// querying and the rate of false positives.
|
||||
//
|
||||
class ARROW_ACERO_EXPORT BlockedBloomFilter {
|
||||
friend class BloomFilterBuilder_SingleThreaded;
|
||||
friend class BloomFilterBuilder_Parallel;
|
||||
|
||||
public:
|
||||
BlockedBloomFilter() : log_num_blocks_(0), num_blocks_(0), blocks_(NULLPTR) {}
|
||||
|
||||
inline bool Find(uint64_t hash) const {
|
||||
uint64_t m = mask(hash);
|
||||
uint64_t b = blocks_[block_id(hash)];
|
||||
return (b & m) == m;
|
||||
}
|
||||
|
||||
// Uses SIMD if available for smaller Bloom filters.
|
||||
// Uses memory prefetching for larger Bloom filters.
|
||||
//
|
||||
void Find(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes,
|
||||
uint8_t* result_bit_vector, bool enable_prefetch = true) const;
|
||||
void Find(int64_t hardware_flags, int64_t num_rows, const uint64_t* hashes,
|
||||
uint8_t* result_bit_vector, bool enable_prefetch = true) const;
|
||||
|
||||
int log_num_blocks() const { return log_num_blocks_; }
|
||||
|
||||
int NumHashBitsUsed() const;
|
||||
|
||||
bool IsSameAs(const BlockedBloomFilter* other) const;
|
||||
|
||||
int64_t NumBitsSet() const;
|
||||
|
||||
// Folding of a block Bloom filter after the initial version
|
||||
// has been built.
|
||||
//
|
||||
// One of the parameters for creation of Bloom filter is the number
|
||||
// of bits allocated for it. The more bits allocated, the lower the
|
||||
// probability of false positives. A good heuristic is to aim for
|
||||
// half of the bits set in the constructed Bloom filter. This should
|
||||
// result in a good trade off between size (and following cost of
|
||||
// memory accesses) and false positives rate.
|
||||
//
|
||||
// There might have been many duplicate keys in the input provided
|
||||
// to Bloom filter builder. In that case the resulting bit vector
|
||||
// would be more sparse then originally intended. It is possible to
|
||||
// easily correct that and cut in half the size of Bloom filter
|
||||
// after it has already been constructed. The process to do that is
|
||||
// approximately equal to OR-ing bits from upper and lower half (the
|
||||
// way we address these bits when inserting or querying a hash makes
|
||||
// such folding in half possible).
|
||||
//
|
||||
// We will keep folding as long as the fraction of bits set is less
|
||||
// than 1/4. The resulting bit vector density should be in the [1/4,
|
||||
// 1/2) range.
|
||||
//
|
||||
void Fold();
|
||||
|
||||
private:
|
||||
Status CreateEmpty(int64_t num_rows_to_insert, MemoryPool* pool);
|
||||
|
||||
inline void Insert(uint64_t hash) {
|
||||
uint64_t m = mask(hash);
|
||||
uint64_t& b = blocks_[block_id(hash)];
|
||||
b |= m;
|
||||
}
|
||||
|
||||
void Insert(int64_t hardware_flags, int64_t num_rows, const uint32_t* hashes);
|
||||
void Insert(int64_t hardware_flags, int64_t num_rows, const uint64_t* hashes);
|
||||
|
||||
inline uint64_t mask(uint64_t hash) const {
|
||||
// The lowest bits of hash are used to pick mask index.
|
||||
//
|
||||
int mask_id = static_cast<int>(hash & (BloomFilterMasks::kNumMasks - 1));
|
||||
uint64_t result = masks_.mask(mask_id);
|
||||
|
||||
// The next set of hash bits is used to pick the amount of bit
|
||||
// rotation of the mask.
|
||||
//
|
||||
int rotation = (hash >> BloomFilterMasks::kLogNumMasks) & 63;
|
||||
result = ROTL64(result, rotation);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
inline int64_t block_id(uint64_t hash) const {
|
||||
// The next set of hash bits following the bits used to select a
|
||||
// mask is used to pick block id (index of 64-bit word in a bit
|
||||
// vector).
|
||||
//
|
||||
return (hash >> (BloomFilterMasks::kLogNumMasks + 6)) & (num_blocks_ - 1);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline void InsertImp(int64_t num_rows, const T* hashes);
|
||||
|
||||
template <typename T>
|
||||
inline void FindImp(int64_t num_rows, const T* hashes, uint8_t* result_bit_vector,
|
||||
bool enable_prefetch) const;
|
||||
|
||||
void SingleFold(int num_folds);
|
||||
|
||||
#if defined(ARROW_HAVE_RUNTIME_AVX2)
|
||||
inline __m256i mask_avx2(__m256i hash) const;
|
||||
inline __m256i block_id_avx2(__m256i hash) const;
|
||||
int64_t Insert_avx2(int64_t num_rows, const uint32_t* hashes);
|
||||
int64_t Insert_avx2(int64_t num_rows, const uint64_t* hashes);
|
||||
template <typename T>
|
||||
int64_t InsertImp_avx2(int64_t num_rows, const T* hashes);
|
||||
int64_t Find_avx2(int64_t num_rows, const uint32_t* hashes,
|
||||
uint8_t* result_bit_vector) const;
|
||||
int64_t Find_avx2(int64_t num_rows, const uint64_t* hashes,
|
||||
uint8_t* result_bit_vector) const;
|
||||
template <typename T>
|
||||
int64_t FindImp_avx2(int64_t num_rows, const T* hashes,
|
||||
uint8_t* result_bit_vector) const;
|
||||
#endif
|
||||
|
||||
bool UsePrefetch() const {
|
||||
return num_blocks_ * sizeof(uint64_t) > kPrefetchLimitBytes;
|
||||
}
|
||||
|
||||
static constexpr int64_t kPrefetchLimitBytes = 256 * 1024;
|
||||
|
||||
static BloomFilterMasks masks_;
|
||||
|
||||
// Total number of bits used by block Bloom filter must be a power
|
||||
// of 2.
|
||||
//
|
||||
int log_num_blocks_;
|
||||
int64_t num_blocks_;
|
||||
|
||||
// Buffer allocated to store an array of power of 2 64-bit blocks.
|
||||
//
|
||||
std::shared_ptr<Buffer> buf_;
|
||||
// Pointer to mutable data owned by Buffer
|
||||
//
|
||||
uint64_t* blocks_;
|
||||
};
|
||||
|
||||
// We have two separate implementations of building a Bloom filter, multi-threaded and
|
||||
// single-threaded.
|
||||
//
|
||||
// Single threaded version is useful in two ways:
|
||||
// a) It allows to verify parallel implementation in tests (the single threaded one is
|
||||
// simpler and can be used as the source of truth).
|
||||
// b) It is preferred for small and medium size Bloom filters, because it skips extra
|
||||
// synchronization related steps from parallel variant (partitioning and taking locks).
|
||||
//
|
||||
enum class BloomFilterBuildStrategy {
|
||||
SINGLE_THREADED = 0,
|
||||
PARALLEL = 1,
|
||||
};
|
||||
|
||||
class ARROW_ACERO_EXPORT BloomFilterBuilder {
|
||||
public:
|
||||
virtual ~BloomFilterBuilder() = default;
|
||||
virtual Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool,
|
||||
int64_t num_rows, int64_t num_batches,
|
||||
BlockedBloomFilter* build_target) = 0;
|
||||
virtual int64_t num_tasks() const { return 0; }
|
||||
virtual Status PushNextBatch(size_t thread_index, int64_t num_rows,
|
||||
const uint32_t* hashes) = 0;
|
||||
virtual Status PushNextBatch(size_t thread_index, int64_t num_rows,
|
||||
const uint64_t* hashes) = 0;
|
||||
virtual void CleanUp() {}
|
||||
static std::unique_ptr<BloomFilterBuilder> Make(BloomFilterBuildStrategy strategy);
|
||||
};
|
||||
|
||||
class ARROW_ACERO_EXPORT BloomFilterBuilder_SingleThreaded : public BloomFilterBuilder {
|
||||
public:
|
||||
Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool,
|
||||
int64_t num_rows, int64_t num_batches,
|
||||
BlockedBloomFilter* build_target) override;
|
||||
|
||||
Status PushNextBatch(size_t /*thread_index*/, int64_t num_rows,
|
||||
const uint32_t* hashes) override;
|
||||
|
||||
Status PushNextBatch(size_t /*thread_index*/, int64_t num_rows,
|
||||
const uint64_t* hashes) override;
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
void PushNextBatchImp(int64_t num_rows, const T* hashes);
|
||||
|
||||
int64_t hardware_flags_;
|
||||
BlockedBloomFilter* build_target_;
|
||||
};
|
||||
|
||||
class ARROW_ACERO_EXPORT BloomFilterBuilder_Parallel : public BloomFilterBuilder {
|
||||
public:
|
||||
Status Begin(size_t num_threads, int64_t hardware_flags, MemoryPool* pool,
|
||||
int64_t num_rows, int64_t num_batches,
|
||||
BlockedBloomFilter* build_target) override;
|
||||
|
||||
Status PushNextBatch(size_t thread_id, int64_t num_rows,
|
||||
const uint32_t* hashes) override;
|
||||
|
||||
Status PushNextBatch(size_t thread_id, int64_t num_rows,
|
||||
const uint64_t* hashes) override;
|
||||
|
||||
void CleanUp() override;
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
void PushNextBatchImp(size_t thread_id, int64_t num_rows, const T* hashes);
|
||||
|
||||
int64_t hardware_flags_;
|
||||
BlockedBloomFilter* build_target_;
|
||||
int log_num_prtns_;
|
||||
struct ThreadLocalState {
|
||||
std::vector<uint32_t> partitioned_hashes_32;
|
||||
std::vector<uint64_t> partitioned_hashes_64;
|
||||
std::vector<uint16_t> partition_ranges;
|
||||
std::vector<int> unprocessed_partition_ids;
|
||||
};
|
||||
std::vector<ThreadLocalState> thread_local_states_;
|
||||
PartitionLocks prtn_locks_;
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,819 @@
|
||||
// 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 <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/type_fwd.h"
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/compute/api_vector.h"
|
||||
#include "arrow/compute/exec.h"
|
||||
#include "arrow/compute/ordering.h"
|
||||
#include "arrow/type_fwd.h"
|
||||
#include "arrow/util/future.h"
|
||||
#include "arrow/util/macros.h"
|
||||
#include "arrow/util/tracing.h"
|
||||
#include "arrow/util/type_fwd.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
using compute::ExecBatch;
|
||||
using compute::ExecContext;
|
||||
using compute::FunctionRegistry;
|
||||
using compute::GetFunctionRegistry;
|
||||
using compute::Ordering;
|
||||
using compute::threaded_exec_context;
|
||||
|
||||
namespace acero {
|
||||
|
||||
/// \addtogroup acero-internals
|
||||
/// @{
|
||||
|
||||
class ARROW_ACERO_EXPORT ExecPlan : public std::enable_shared_from_this<ExecPlan> {
|
||||
public:
|
||||
// This allows operators to rely on signed 16-bit indices
|
||||
static const uint32_t kMaxBatchSize = 1 << 15;
|
||||
using NodeVector = std::vector<ExecNode*>;
|
||||
|
||||
virtual ~ExecPlan() = default;
|
||||
|
||||
QueryContext* query_context();
|
||||
|
||||
/// \brief retrieve the nodes in the plan
|
||||
const NodeVector& nodes() const;
|
||||
|
||||
/// Make an empty exec plan
|
||||
static Result<std::shared_ptr<ExecPlan>> Make(
|
||||
QueryOptions options, ExecContext exec_context = *threaded_exec_context(),
|
||||
std::shared_ptr<const KeyValueMetadata> metadata = NULLPTR);
|
||||
|
||||
static Result<std::shared_ptr<ExecPlan>> Make(
|
||||
ExecContext exec_context = *threaded_exec_context(),
|
||||
std::shared_ptr<const KeyValueMetadata> metadata = NULLPTR);
|
||||
|
||||
static Result<std::shared_ptr<ExecPlan>> Make(
|
||||
QueryOptions options, ExecContext* exec_context,
|
||||
std::shared_ptr<const KeyValueMetadata> metadata = NULLPTR);
|
||||
|
||||
static Result<std::shared_ptr<ExecPlan>> Make(
|
||||
ExecContext* exec_context,
|
||||
std::shared_ptr<const KeyValueMetadata> metadata = NULLPTR);
|
||||
|
||||
ExecNode* AddNode(std::unique_ptr<ExecNode> node);
|
||||
|
||||
template <typename Node, typename... Args>
|
||||
Node* EmplaceNode(Args&&... args) {
|
||||
std::unique_ptr<Node> node{new Node{std::forward<Args>(args)...}};
|
||||
auto out = node.get();
|
||||
AddNode(std::move(node));
|
||||
return out;
|
||||
}
|
||||
|
||||
Status Validate();
|
||||
|
||||
/// \brief Start producing on all nodes
|
||||
///
|
||||
/// Nodes are started in reverse topological order, such that any node
|
||||
/// is started before all of its inputs.
|
||||
void StartProducing();
|
||||
|
||||
/// \brief Stop producing on all nodes
|
||||
///
|
||||
/// Triggers all sources to stop producing new data. In order to cleanly stop the plan
|
||||
/// will continue to run any tasks that are already in progress. The caller should
|
||||
/// still wait for `finished` to complete before destroying the plan.
|
||||
void StopProducing();
|
||||
|
||||
/// \brief A future which will be marked finished when all tasks have finished.
|
||||
Future<> finished();
|
||||
|
||||
/// \brief Return whether the plan has non-empty metadata
|
||||
bool HasMetadata() const;
|
||||
|
||||
/// \brief Return the plan's attached metadata
|
||||
std::shared_ptr<const KeyValueMetadata> metadata() const;
|
||||
|
||||
std::string ToString() const;
|
||||
};
|
||||
|
||||
// Acero can be extended by providing custom implementations of ExecNode. The methods
|
||||
// below are documented in detail and provide careful instruction on how to fulfill the
|
||||
// ExecNode contract. It's suggested you familiarize yourself with the Acero
|
||||
// documentation in the C++ user guide.
|
||||
class ARROW_ACERO_EXPORT ExecNode {
|
||||
public:
|
||||
using NodeVector = std::vector<ExecNode*>;
|
||||
|
||||
virtual ~ExecNode() = default;
|
||||
|
||||
virtual const char* kind_name() const = 0;
|
||||
|
||||
// The number of inputs expected by this node
|
||||
int num_inputs() const { return static_cast<int>(inputs_.size()); }
|
||||
|
||||
/// This node's predecessors in the exec plan
|
||||
const NodeVector& inputs() const { return inputs_; }
|
||||
|
||||
/// True if the plan has no output schema (is a sink)
|
||||
bool is_sink() const { return !output_schema_; }
|
||||
|
||||
/// \brief Labels identifying the function of each input.
|
||||
const std::vector<std::string>& input_labels() const { return input_labels_; }
|
||||
|
||||
/// This node's successor in the exec plan
|
||||
const ExecNode* output() const { return output_; }
|
||||
|
||||
/// The datatypes for batches produced by this node
|
||||
const std::shared_ptr<Schema>& output_schema() const { return output_schema_; }
|
||||
|
||||
/// This node's exec plan
|
||||
ExecPlan* plan() { return plan_; }
|
||||
|
||||
/// \brief An optional label, for display and debugging
|
||||
///
|
||||
/// There is no guarantee that this value is non-empty or unique.
|
||||
const std::string& label() const { return label_; }
|
||||
void SetLabel(std::string label) { label_ = std::move(label); }
|
||||
|
||||
virtual Status Validate() const;
|
||||
|
||||
/// \brief the ordering of the output batches
|
||||
///
|
||||
/// This does not guarantee the batches will be emitted by this node
|
||||
/// in order. Instead it guarantees that the batches will have their
|
||||
/// ExecBatch::index property set in a way that respects this ordering.
|
||||
///
|
||||
/// In other words, given the ordering {{"x", SortOrder::Ascending}} we
|
||||
/// know that all values of x in a batch with index N will be less than
|
||||
/// or equal to all values of x in a batch with index N+k (assuming k > 0).
|
||||
/// Furthermore, we also know that values will be sorted within a batch.
|
||||
/// Any row N will have a value of x that is less than the value for
|
||||
/// any row N+k.
|
||||
///
|
||||
/// Note that an ordering can be both Ordering::Unordered and Ordering::Implicit.
|
||||
/// A node's output should be marked Ordering::Unordered if the order is
|
||||
/// non-deterministic. For example, a hash-join has no predictable output order.
|
||||
///
|
||||
/// If the ordering is Ordering::Implicit then there is a meaningful order but that
|
||||
/// ordering is not represented by any column in the data. The most common case for
|
||||
/// this is when reading data from an in-memory table. The data has an implicit "row
|
||||
/// order" which is not necessarily represented in the data set.
|
||||
///
|
||||
/// A filter or project node will not modify the ordering. Nothing needs to be done
|
||||
/// other than ensure the index assigned to output batches is the same as the
|
||||
/// input batch that was mapped.
|
||||
///
|
||||
/// Other nodes may introduce order. For example, an order-by node will emit
|
||||
/// a brand new ordering independent of the input ordering.
|
||||
///
|
||||
/// Finally, as described above, such as a hash-join or aggregation may may
|
||||
/// destroy ordering (although these nodes could also choose to establish a
|
||||
/// new ordering based on the hash keys).
|
||||
///
|
||||
/// Some nodes will require an ordering. For example, a fetch node or an
|
||||
/// asof join node will only function if the input data is ordered (for fetch
|
||||
/// it is enough to be implicitly ordered. For an asof join the ordering must
|
||||
/// be explicit and compatible with the on key.)
|
||||
///
|
||||
/// Nodes that maintain ordering should be careful to avoid introducing gaps
|
||||
/// in the batch index. This may require emitting empty batches in order to
|
||||
/// maintain continuity.
|
||||
virtual const Ordering& ordering() const;
|
||||
|
||||
/// Upstream API:
|
||||
/// These functions are called by input nodes that want to inform this node
|
||||
/// about an updated condition (a new input batch or an impending
|
||||
/// end of stream).
|
||||
///
|
||||
/// Implementation rules:
|
||||
/// - these may be called anytime after StartProducing() has succeeded
|
||||
/// (and even during or after StopProducing())
|
||||
/// - these may be called concurrently
|
||||
/// - these are allowed to call back into PauseProducing(), ResumeProducing()
|
||||
/// and StopProducing()
|
||||
|
||||
/// Transfer input batch to ExecNode
|
||||
///
|
||||
/// A node will typically perform some kind of operation on the batch
|
||||
/// and then call InputReceived on its outputs with the result.
|
||||
///
|
||||
/// Other nodes may need to accumulate some number of inputs before any
|
||||
/// output can be produced. These nodes will add the batch to some kind
|
||||
/// of in-memory accumulation queue and return.
|
||||
virtual Status InputReceived(ExecNode* input, ExecBatch batch) = 0;
|
||||
|
||||
/// Mark the inputs finished after the given number of batches.
|
||||
///
|
||||
/// This may be called before all inputs are received. This simply fixes
|
||||
/// the total number of incoming batches for an input, so that the ExecNode
|
||||
/// knows when it has received all input, regardless of order.
|
||||
virtual Status InputFinished(ExecNode* input, int total_batches) = 0;
|
||||
|
||||
/// \brief Perform any needed initialization
|
||||
///
|
||||
/// This hook performs any actions in between creation of ExecPlan and the call to
|
||||
/// StartProducing. An example could be Bloom filter pushdown. The order of ExecNodes
|
||||
/// that executes this method is undefined, but the calls are made synchronously.
|
||||
///
|
||||
/// At this point a node can rely on all inputs & outputs (and the input schemas)
|
||||
/// being well defined.
|
||||
virtual Status Init();
|
||||
|
||||
/// Lifecycle API:
|
||||
/// - start / stop to initiate and terminate production
|
||||
/// - pause / resume to apply backpressure
|
||||
///
|
||||
/// Implementation rules:
|
||||
/// - StartProducing() should not recurse into the inputs, as it is
|
||||
/// handled by ExecPlan::StartProducing()
|
||||
/// - PauseProducing(), ResumeProducing(), StopProducing() may be called
|
||||
/// concurrently, potentially even before the call to StartProducing
|
||||
/// has finished.
|
||||
/// - PauseProducing(), ResumeProducing(), StopProducing() may be called
|
||||
/// by the downstream nodes' InputReceived(), InputFinished() methods
|
||||
///
|
||||
/// StopProducing may be called due to an error, by the user (e.g. cancel), or
|
||||
/// because a node has all the data it needs (e.g. limit, top-k on sorted data).
|
||||
/// This means the method may be called multiple times and we have the following
|
||||
/// additional rules
|
||||
/// - StopProducing() must be idempotent
|
||||
/// - StopProducing() must be forwarded to inputs (this is needed for the limit/top-k
|
||||
/// case because we may not be stopping the entire plan)
|
||||
|
||||
// Right now, since synchronous calls happen in both directions (input to
|
||||
// output and then output to input), a node must be careful to be reentrant
|
||||
// against synchronous calls from its output, *and* also concurrent calls from
|
||||
// other threads. The most reliable solution is to update the internal state
|
||||
// first, and notify outputs only at the end.
|
||||
//
|
||||
// Concurrent calls to PauseProducing and ResumeProducing can be hard to sequence
|
||||
// as they may travel at different speeds through the plan.
|
||||
//
|
||||
// For example, consider a resume that comes quickly after a pause. If the source
|
||||
// receives the resume before the pause the source may think the destination is full
|
||||
// and halt production which would lead to deadlock.
|
||||
//
|
||||
// To resolve this a counter is sent for all calls to pause/resume. Only the call with
|
||||
// the highest counter value is valid. So if a call to PauseProducing(5) comes after
|
||||
// a call to ResumeProducing(6) then the source should continue producing.
|
||||
|
||||
/// \brief Start producing
|
||||
///
|
||||
/// This must only be called once.
|
||||
///
|
||||
/// This is typically called automatically by ExecPlan::StartProducing().
|
||||
virtual Status StartProducing() = 0;
|
||||
|
||||
/// \brief Pause producing temporarily
|
||||
///
|
||||
/// \param output Pointer to the output that is full
|
||||
/// \param counter Counter used to sequence calls to pause/resume
|
||||
///
|
||||
/// This call is a hint that an output node is currently not willing
|
||||
/// to receive data.
|
||||
///
|
||||
/// This may be called any number of times.
|
||||
/// However, the node is still free to produce data (which may be difficult
|
||||
/// to prevent anyway if data is produced using multiple threads).
|
||||
virtual void PauseProducing(ExecNode* output, int32_t counter) = 0;
|
||||
|
||||
/// \brief Resume producing after a temporary pause
|
||||
///
|
||||
/// \param output Pointer to the output that is now free
|
||||
/// \param counter Counter used to sequence calls to pause/resume
|
||||
///
|
||||
/// This call is a hint that an output node is willing to receive data again.
|
||||
///
|
||||
/// This may be called any number of times.
|
||||
virtual void ResumeProducing(ExecNode* output, int32_t counter) = 0;
|
||||
|
||||
/// \brief Stop producing new data
|
||||
///
|
||||
/// If this node is a source then the source should stop generating data
|
||||
/// as quickly as possible. If this node is not a source then there is typically
|
||||
/// nothing that needs to be done although a node may choose to start ignoring incoming
|
||||
/// data.
|
||||
///
|
||||
/// This method will be called when an error occurs in the plan
|
||||
/// This method may also be called by the user if they wish to end a plan early
|
||||
/// Finally, this method may be called if a node determines it no longer needs any more
|
||||
/// input (for example, a limit node).
|
||||
///
|
||||
/// This method may be called multiple times.
|
||||
///
|
||||
/// This is not a pause. There will be no way to start the source again after this has
|
||||
/// been called.
|
||||
virtual Status StopProducing();
|
||||
|
||||
std::string ToString(int indent = 0) const;
|
||||
|
||||
protected:
|
||||
ExecNode(ExecPlan* plan, NodeVector inputs, std::vector<std::string> input_labels,
|
||||
std::shared_ptr<Schema> output_schema);
|
||||
|
||||
virtual Status StopProducingImpl() = 0;
|
||||
|
||||
/// Provide extra info to include in the string representation.
|
||||
virtual std::string ToStringExtra(int indent = 0) const;
|
||||
|
||||
std::atomic<bool> stopped_;
|
||||
ExecPlan* plan_;
|
||||
std::string label_;
|
||||
|
||||
NodeVector inputs_;
|
||||
std::vector<std::string> input_labels_;
|
||||
|
||||
std::shared_ptr<Schema> output_schema_;
|
||||
ExecNode* output_ = NULLPTR;
|
||||
};
|
||||
|
||||
/// \brief An extensible registry for factories of ExecNodes
|
||||
class ARROW_ACERO_EXPORT ExecFactoryRegistry {
|
||||
public:
|
||||
using Factory = std::function<Result<ExecNode*>(ExecPlan*, std::vector<ExecNode*>,
|
||||
const ExecNodeOptions&)>;
|
||||
|
||||
virtual ~ExecFactoryRegistry() = default;
|
||||
|
||||
/// \brief Get the named factory from this registry
|
||||
///
|
||||
/// will raise if factory_name is not found
|
||||
virtual Result<Factory> GetFactory(const std::string& factory_name) = 0;
|
||||
|
||||
/// \brief Add a factory to this registry with the provided name
|
||||
///
|
||||
/// will raise if factory_name is already in the registry
|
||||
virtual Status AddFactory(std::string factory_name, Factory factory) = 0;
|
||||
};
|
||||
|
||||
/// The default registry, which includes built-in factories.
|
||||
ARROW_ACERO_EXPORT
|
||||
ExecFactoryRegistry* default_exec_factory_registry();
|
||||
|
||||
/// \brief Construct an ExecNode using the named factory
|
||||
inline Result<ExecNode*> MakeExecNode(
|
||||
const std::string& factory_name, ExecPlan* plan, std::vector<ExecNode*> inputs,
|
||||
const ExecNodeOptions& options,
|
||||
ExecFactoryRegistry* registry = default_exec_factory_registry()) {
|
||||
ARROW_ASSIGN_OR_RAISE(auto factory, registry->GetFactory(factory_name));
|
||||
return factory(plan, std::move(inputs), options);
|
||||
}
|
||||
|
||||
/// @}
|
||||
|
||||
/// \addtogroup acero-api
|
||||
/// @{
|
||||
|
||||
/// \brief Helper class for declaring execution nodes
|
||||
///
|
||||
/// A Declaration represents an unconstructed ExecNode (and potentially an entire graph
|
||||
/// since its inputs may also be Declarations)
|
||||
///
|
||||
/// A Declaration can be converted to a plan and executed using one of the
|
||||
/// DeclarationToXyz methods.
|
||||
///
|
||||
/// For more direct control, a Declaration can be added to an existing execution
|
||||
/// plan with Declaration::AddToPlan, which will recursively construct any inputs as
|
||||
/// necessary.
|
||||
struct ARROW_ACERO_EXPORT Declaration {
|
||||
using Input = std::variant<ExecNode*, Declaration>;
|
||||
|
||||
Declaration() {}
|
||||
|
||||
/// \brief construct a declaration
|
||||
/// \param factory_name the name of the exec node to construct. The node must have
|
||||
/// been added to the exec node registry with this name.
|
||||
/// \param inputs the inputs to the node, these should be other declarations
|
||||
/// \param options options that control the behavior of the node. You must use
|
||||
/// the appropriate subclass. For example, if `factory_name` is
|
||||
/// "project" then `options` should be ProjectNodeOptions.
|
||||
/// \param label a label to give the node. Can be used to distinguish it from other
|
||||
/// nodes of the same type in the plan.
|
||||
Declaration(std::string factory_name, std::vector<Input> inputs,
|
||||
std::shared_ptr<ExecNodeOptions> options, std::string label)
|
||||
: factory_name{std::move(factory_name)},
|
||||
inputs{std::move(inputs)},
|
||||
options{std::move(options)},
|
||||
label{std::move(label)} {}
|
||||
|
||||
template <typename Options>
|
||||
Declaration(std::string factory_name, std::vector<Input> inputs, Options options,
|
||||
std::string label)
|
||||
: Declaration{std::move(factory_name), std::move(inputs),
|
||||
std::shared_ptr<ExecNodeOptions>(
|
||||
std::make_shared<Options>(std::move(options))),
|
||||
std::move(label)} {}
|
||||
|
||||
template <typename Options>
|
||||
Declaration(std::string factory_name, std::vector<Input> inputs, Options options)
|
||||
: Declaration{std::move(factory_name), std::move(inputs), std::move(options),
|
||||
/*label=*/""} {}
|
||||
|
||||
template <typename Options>
|
||||
Declaration(std::string factory_name, Options options)
|
||||
: Declaration{std::move(factory_name), {}, std::move(options), /*label=*/""} {}
|
||||
|
||||
template <typename Options>
|
||||
Declaration(std::string factory_name, Options options, std::string label)
|
||||
: Declaration{std::move(factory_name), {}, std::move(options), std::move(label)} {}
|
||||
|
||||
/// \brief Convenience factory for the common case of a simple sequence of nodes.
|
||||
///
|
||||
/// Each of decls will be appended to the inputs of the subsequent declaration,
|
||||
/// and the final modified declaration will be returned.
|
||||
///
|
||||
/// Without this convenience factory, constructing a sequence would require explicit,
|
||||
/// difficult-to-read nesting:
|
||||
///
|
||||
/// Declaration{"n3",
|
||||
/// {
|
||||
/// Declaration{"n2",
|
||||
/// {
|
||||
/// Declaration{"n1",
|
||||
/// {
|
||||
/// Declaration{"n0", N0Opts{}},
|
||||
/// },
|
||||
/// N1Opts{}},
|
||||
/// },
|
||||
/// N2Opts{}},
|
||||
/// },
|
||||
/// N3Opts{}};
|
||||
///
|
||||
/// An equivalent Declaration can be constructed more tersely using Sequence:
|
||||
///
|
||||
/// Declaration::Sequence({
|
||||
/// {"n0", N0Opts{}},
|
||||
/// {"n1", N1Opts{}},
|
||||
/// {"n2", N2Opts{}},
|
||||
/// {"n3", N3Opts{}},
|
||||
/// });
|
||||
static Declaration Sequence(std::vector<Declaration> decls);
|
||||
|
||||
/// \brief add the declaration to an already created execution plan
|
||||
/// \param plan the plan to add the node to
|
||||
/// \param registry the registry to use to lookup the node factory
|
||||
///
|
||||
/// This method will recursively call AddToPlan on all of the declaration's inputs.
|
||||
/// This method is only for advanced use when the DeclarationToXyz methods are not
|
||||
/// sufficient.
|
||||
///
|
||||
/// \return the instantiated execution node
|
||||
Result<ExecNode*> AddToPlan(ExecPlan* plan, ExecFactoryRegistry* registry =
|
||||
default_exec_factory_registry()) const;
|
||||
|
||||
// Validate a declaration
|
||||
bool IsValid(ExecFactoryRegistry* registry = default_exec_factory_registry()) const;
|
||||
|
||||
/// \brief the name of the factory to use when creating a node
|
||||
std::string factory_name;
|
||||
/// \brief the declarations's inputs
|
||||
std::vector<Input> inputs;
|
||||
/// \brief options to control the behavior of the node
|
||||
std::shared_ptr<ExecNodeOptions> options;
|
||||
/// \brief a label to give the node in the plan
|
||||
std::string label;
|
||||
};
|
||||
|
||||
/// \brief How to handle unaligned buffers
|
||||
enum class UnalignedBufferHandling { kWarn, kIgnore, kReallocate, kError };
|
||||
|
||||
/// \brief get the default behavior of unaligned buffer handling
|
||||
///
|
||||
/// This is configurable via the ACERO_ALIGNMENT_HANDLING environment variable which
|
||||
/// can be set to "warn", "ignore", "reallocate", or "error". If the environment
|
||||
/// variable is not set, or is set to an invalid value, this will return kWarn
|
||||
UnalignedBufferHandling GetDefaultUnalignedBufferHandling();
|
||||
|
||||
/// \brief plan-wide options that can be specified when executing an execution plan
|
||||
struct ARROW_ACERO_EXPORT QueryOptions {
|
||||
/// \brief Should the plan use a legacy batching strategy
|
||||
///
|
||||
/// This is currently in place only to support the Scanner::ToTable
|
||||
/// method. This method relies on batch indices from the scanner
|
||||
/// remaining consistent. This is impractical in the ExecPlan which
|
||||
/// might slice batches as needed (e.g. for a join)
|
||||
///
|
||||
/// However, it still works for simple plans and this is the only way
|
||||
/// we have at the moment for maintaining implicit order.
|
||||
bool use_legacy_batching = false;
|
||||
|
||||
/// If the output has a meaningful order then sequence the output of the plan
|
||||
///
|
||||
/// The default behavior (std::nullopt) will sequence output batches if there
|
||||
/// is a meaningful ordering in the final node and will emit batches immediately
|
||||
/// otherwise.
|
||||
///
|
||||
/// If explicitly set to true then plan execution will fail if there is no
|
||||
/// meaningful ordering. This can be useful to validate a query that should
|
||||
/// be emitting ordered results.
|
||||
///
|
||||
/// If explicitly set to false then batches will be emit immediately even if there
|
||||
/// is a meaningful ordering. This could cause batches to be emit out of order but
|
||||
/// may offer a small decrease to latency.
|
||||
std::optional<bool> sequence_output = std::nullopt;
|
||||
|
||||
/// \brief should the plan use multiple background threads for CPU-intensive work
|
||||
///
|
||||
/// If this is false then all CPU work will be done on the calling thread. I/O tasks
|
||||
/// will still happen on the I/O executor and may be multi-threaded (but should not use
|
||||
/// significant CPU resources).
|
||||
///
|
||||
/// Will be ignored if custom_cpu_executor is set
|
||||
bool use_threads = true;
|
||||
|
||||
/// \brief custom executor to use for CPU-intensive work
|
||||
///
|
||||
/// Must be null or remain valid for the duration of the plan. If this is null then
|
||||
/// a default thread pool will be chosen whose behavior will be controlled by
|
||||
/// the `use_threads` option.
|
||||
::arrow::internal::Executor* custom_cpu_executor = NULLPTR;
|
||||
|
||||
/// \brief custom executor to use for IO work
|
||||
///
|
||||
/// Must be null or remain valid for the duration of the plan. If this is null then
|
||||
/// the global io thread pool will be chosen whose behavior will be controlled by
|
||||
/// the "ARROW_IO_THREADS" environment.
|
||||
::arrow::internal::Executor* custom_io_executor = NULLPTR;
|
||||
|
||||
/// \brief a memory pool to use for allocations
|
||||
///
|
||||
/// Must remain valid for the duration of the plan.
|
||||
MemoryPool* memory_pool = default_memory_pool();
|
||||
|
||||
/// \brief a function registry to use for the plan
|
||||
///
|
||||
/// Must remain valid for the duration of the plan.
|
||||
FunctionRegistry* function_registry = GetFunctionRegistry();
|
||||
/// \brief the names of the output columns
|
||||
///
|
||||
/// If this is empty then names will be generated based on the input columns
|
||||
///
|
||||
/// If set then the number of names must equal the number of output columns
|
||||
std::vector<std::string> field_names;
|
||||
|
||||
/// \brief Policy for unaligned buffers in source data
|
||||
///
|
||||
/// Various compute functions and acero internals will type pun array
|
||||
/// buffers from uint8_t* to some kind of value type (e.g. we might
|
||||
/// cast to int32_t* to add two int32 arrays)
|
||||
///
|
||||
/// If the buffer is poorly aligned (e.g. an int32 array is not aligned
|
||||
/// on a 4-byte boundary) then this is technically undefined behavior in C++.
|
||||
/// However, most modern compilers and CPUs are fairly tolerant of this
|
||||
/// behavior and nothing bad (beyond a small hit to performance) is likely
|
||||
/// to happen.
|
||||
///
|
||||
/// Note that this only applies to source buffers. All buffers allocated internally
|
||||
/// by Acero will be suitably aligned.
|
||||
///
|
||||
/// If this field is set to kWarn then Acero will check if any buffers are unaligned
|
||||
/// and, if they are, will emit a warning.
|
||||
///
|
||||
/// If this field is set to kReallocate then Acero will allocate a new, suitably aligned
|
||||
/// buffer and copy the contents from the old buffer into this new buffer.
|
||||
///
|
||||
/// If this field is set to kError then Acero will gracefully abort the plan instead.
|
||||
///
|
||||
/// If this field is set to kIgnore then Acero will not even check if the buffers are
|
||||
/// unaligned.
|
||||
///
|
||||
/// If this field is not set then it will be treated as kWarn unless overridden
|
||||
/// by the ACERO_ALIGNMENT_HANDLING environment variable
|
||||
std::optional<UnalignedBufferHandling> unaligned_buffer_handling;
|
||||
};
|
||||
|
||||
/// \brief Calculate the output schema of a declaration
|
||||
///
|
||||
/// This does not actually execute the plan. This operation may fail if the
|
||||
/// declaration represents an invalid plan (e.g. a project node with multiple inputs)
|
||||
///
|
||||
/// \param declaration A declaration describing an execution plan
|
||||
/// \param function_registry The function registry to use for function execution. If null
|
||||
/// then the default function registry will be used.
|
||||
///
|
||||
/// \return the schema that batches would have after going through the execution plan
|
||||
ARROW_ACERO_EXPORT Result<std::shared_ptr<Schema>> DeclarationToSchema(
|
||||
const Declaration& declaration, FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
/// \brief Create a string representation of a plan
|
||||
///
|
||||
/// This representation is for debug purposes only.
|
||||
///
|
||||
/// Conversion to a string may fail if the declaration represents an
|
||||
/// invalid plan.
|
||||
///
|
||||
/// Use Substrait for complete serialization of plans
|
||||
///
|
||||
/// \param declaration A declaration describing an execution plan
|
||||
/// \param function_registry The function registry to use for function execution. If null
|
||||
/// then the default function registry will be used.
|
||||
///
|
||||
/// \return a string representation of the plan suitable for debugging output
|
||||
ARROW_ACERO_EXPORT Result<std::string> DeclarationToString(
|
||||
const Declaration& declaration, FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
/// \brief Utility method to run a declaration and collect the results into a table
|
||||
///
|
||||
/// \param declaration A declaration describing the plan to run
|
||||
/// \param use_threads If `use_threads` is false then all CPU work will be done on the
|
||||
/// calling thread. I/O tasks will still happen on the I/O executor
|
||||
/// and may be multi-threaded (but should not use significant CPU
|
||||
/// resources).
|
||||
/// \param memory_pool The memory pool to use for allocations made while running the plan.
|
||||
/// \param function_registry The function registry to use for function execution. If null
|
||||
/// then the default function registry will be used.
|
||||
///
|
||||
/// This method will add a sink node to the declaration to collect results into a
|
||||
/// table. It will then create an ExecPlan from the declaration, start the exec plan,
|
||||
/// block until the plan has finished, and return the created table.
|
||||
ARROW_ACERO_EXPORT Result<std::shared_ptr<Table>> DeclarationToTable(
|
||||
Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
ARROW_ACERO_EXPORT Result<std::shared_ptr<Table>> DeclarationToTable(
|
||||
Declaration declaration, QueryOptions query_options);
|
||||
|
||||
/// \brief Asynchronous version of \see DeclarationToTable
|
||||
///
|
||||
/// \param declaration A declaration describing the plan to run
|
||||
/// \param use_threads The behavior of use_threads is slightly different than the
|
||||
/// synchronous version since we cannot run synchronously on the
|
||||
/// calling thread. Instead, if use_threads=false then a new thread
|
||||
/// pool will be created with a single thread and this will be used for
|
||||
/// all compute work.
|
||||
/// \param memory_pool The memory pool to use for allocations made while running the plan.
|
||||
/// \param function_registry The function registry to use for function execution. If null
|
||||
/// then the default function registry will be used.
|
||||
ARROW_ACERO_EXPORT Future<std::shared_ptr<Table>> DeclarationToTableAsync(
|
||||
Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
/// \brief Overload of \see DeclarationToTableAsync accepting a custom exec context
|
||||
///
|
||||
/// The executor must be specified (cannot be null) and must be kept alive until the
|
||||
/// returned future finishes.
|
||||
ARROW_ACERO_EXPORT Future<std::shared_ptr<Table>> DeclarationToTableAsync(
|
||||
Declaration declaration, ExecContext custom_exec_context);
|
||||
|
||||
/// \brief a collection of exec batches with a common schema
|
||||
struct BatchesWithCommonSchema {
|
||||
std::vector<ExecBatch> batches;
|
||||
std::shared_ptr<Schema> schema;
|
||||
};
|
||||
|
||||
/// \brief Utility method to run a declaration and collect the results into ExecBatch
|
||||
/// vector
|
||||
///
|
||||
/// \see DeclarationToTable for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Result<BatchesWithCommonSchema> DeclarationToExecBatches(
|
||||
Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
ARROW_ACERO_EXPORT Result<BatchesWithCommonSchema> DeclarationToExecBatches(
|
||||
Declaration declaration, QueryOptions query_options);
|
||||
|
||||
/// \brief Asynchronous version of \see DeclarationToExecBatches
|
||||
///
|
||||
/// \see DeclarationToTableAsync for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Future<BatchesWithCommonSchema> DeclarationToExecBatchesAsync(
|
||||
Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
/// \brief Overload of \see DeclarationToExecBatchesAsync accepting a custom exec context
|
||||
///
|
||||
/// \see DeclarationToTableAsync for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Future<BatchesWithCommonSchema> DeclarationToExecBatchesAsync(
|
||||
Declaration declaration, ExecContext custom_exec_context);
|
||||
|
||||
/// \brief Utility method to run a declaration and collect the results into a vector
|
||||
///
|
||||
/// \see DeclarationToTable for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Result<std::vector<std::shared_ptr<RecordBatch>>> DeclarationToBatches(
|
||||
Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
ARROW_ACERO_EXPORT Result<std::vector<std::shared_ptr<RecordBatch>>> DeclarationToBatches(
|
||||
Declaration declaration, QueryOptions query_options);
|
||||
|
||||
/// \brief Asynchronous version of \see DeclarationToBatches
|
||||
///
|
||||
/// \see DeclarationToTableAsync for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Future<std::vector<std::shared_ptr<RecordBatch>>>
|
||||
DeclarationToBatchesAsync(Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
/// \brief Overload of \see DeclarationToBatchesAsync accepting a custom exec context
|
||||
///
|
||||
/// \see DeclarationToTableAsync for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Future<std::vector<std::shared_ptr<RecordBatch>>>
|
||||
DeclarationToBatchesAsync(Declaration declaration, ExecContext exec_context);
|
||||
|
||||
/// \brief Utility method to run a declaration and return results as a RecordBatchReader
|
||||
///
|
||||
/// If an exec context is not provided then a default exec context will be used based
|
||||
/// on the value of `use_threads`. If `use_threads` is false then the CPU executor will
|
||||
/// be a serial executor and all CPU work will be done on the calling thread. I/O tasks
|
||||
/// will still happen on the I/O executor and may be multi-threaded.
|
||||
///
|
||||
/// If `use_threads` is false then all CPU work will happen during the calls to
|
||||
/// RecordBatchReader::Next and no CPU work will happen in the background. If
|
||||
/// `use_threads` is true then CPU work will happen on the CPU thread pool and tasks may
|
||||
/// run in between calls to RecordBatchReader::Next. If the returned reader is not
|
||||
/// consumed quickly enough then the plan will eventually pause as the backpressure queue
|
||||
/// fills up.
|
||||
///
|
||||
/// If a custom exec context is provided then the value of `use_threads` will be ignored.
|
||||
///
|
||||
/// The returned RecordBatchReader can be closed early to cancel the computation of record
|
||||
/// batches. In this case, only errors encountered by the computation may be reported. In
|
||||
/// particular, no cancellation error may be reported.
|
||||
ARROW_ACERO_EXPORT Result<std::unique_ptr<RecordBatchReader>> DeclarationToReader(
|
||||
Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
ARROW_ACERO_EXPORT Result<std::unique_ptr<RecordBatchReader>> DeclarationToReader(
|
||||
Declaration declaration, QueryOptions query_options);
|
||||
|
||||
/// \brief Utility method to run a declaration and ignore results
|
||||
///
|
||||
/// This can be useful when the data are consumed as part of the plan itself, for
|
||||
/// example, when the plan ends with a write node.
|
||||
///
|
||||
/// \see DeclarationToTable for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Status
|
||||
DeclarationToStatus(Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
ARROW_ACERO_EXPORT Status DeclarationToStatus(Declaration declaration,
|
||||
QueryOptions query_options);
|
||||
|
||||
/// \brief Asynchronous version of \see DeclarationToStatus
|
||||
///
|
||||
/// This can be useful when the data are consumed as part of the plan itself, for
|
||||
/// example, when the plan ends with a write node.
|
||||
///
|
||||
/// \see DeclarationToTableAsync for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Future<> DeclarationToStatusAsync(
|
||||
Declaration declaration, bool use_threads = true,
|
||||
MemoryPool* memory_pool = default_memory_pool(),
|
||||
FunctionRegistry* function_registry = NULLPTR);
|
||||
|
||||
/// \brief Overload of \see DeclarationToStatusAsync accepting a custom exec context
|
||||
///
|
||||
/// \see DeclarationToTableAsync for details on threading & execution
|
||||
ARROW_ACERO_EXPORT Future<> DeclarationToStatusAsync(Declaration declaration,
|
||||
ExecContext exec_context);
|
||||
|
||||
/// @}
|
||||
|
||||
/// \brief Wrap an ExecBatch generator in a RecordBatchReader.
|
||||
///
|
||||
/// The RecordBatchReader does not impose any ordering on emitted batches.
|
||||
ARROW_ACERO_EXPORT
|
||||
std::shared_ptr<RecordBatchReader> MakeGeneratorReader(
|
||||
std::shared_ptr<Schema>, std::function<Future<std::optional<ExecBatch>>()>,
|
||||
MemoryPool*);
|
||||
|
||||
constexpr int kDefaultBackgroundMaxQ = 32;
|
||||
constexpr int kDefaultBackgroundQRestart = 16;
|
||||
|
||||
/// \brief Make a generator of RecordBatchReaders
|
||||
///
|
||||
/// Useful as a source node for an Exec plan
|
||||
ARROW_ACERO_EXPORT
|
||||
Result<std::function<Future<std::optional<ExecBatch>>()>> MakeReaderGenerator(
|
||||
std::shared_ptr<RecordBatchReader> reader, arrow::internal::Executor* io_executor,
|
||||
int max_q = kDefaultBackgroundMaxQ, int q_restart = kDefaultBackgroundQRestart);
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,75 @@
|
||||
// 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 <functional>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/accumulation_queue.h"
|
||||
#include "arrow/acero/bloom_filter.h"
|
||||
#include "arrow/acero/options.h"
|
||||
#include "arrow/acero/query_context.h"
|
||||
#include "arrow/acero/schema_util.h"
|
||||
#include "arrow/acero/task_util.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/status.h"
|
||||
#include "arrow/type.h"
|
||||
#include "arrow/util/tracing.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
|
||||
using util::AccumulationQueue;
|
||||
|
||||
class ARROW_ACERO_EXPORT HashJoinImpl {
|
||||
public:
|
||||
using OutputBatchCallback = std::function<Status(int64_t, ExecBatch)>;
|
||||
using BuildFinishedCallback = std::function<Status(size_t)>;
|
||||
using FinishedCallback = std::function<Status(int64_t)>;
|
||||
using RegisterTaskGroupCallback = std::function<int(
|
||||
std::function<Status(size_t, int64_t)>, std::function<Status(size_t)>)>;
|
||||
using StartTaskGroupCallback = std::function<Status(int, int64_t)>;
|
||||
using AbortContinuationImpl = std::function<void()>;
|
||||
|
||||
virtual ~HashJoinImpl() = default;
|
||||
virtual Status Init(QueryContext* ctx, JoinType join_type, size_t num_threads,
|
||||
const HashJoinProjectionMaps* proj_map_left,
|
||||
const HashJoinProjectionMaps* proj_map_right,
|
||||
std::vector<JoinKeyCmp> key_cmp, Expression filter,
|
||||
RegisterTaskGroupCallback register_task_group_callback,
|
||||
StartTaskGroupCallback start_task_group_callback,
|
||||
OutputBatchCallback output_batch_callback,
|
||||
FinishedCallback finished_callback) = 0;
|
||||
|
||||
virtual Status BuildHashTable(size_t thread_index, AccumulationQueue batches,
|
||||
BuildFinishedCallback on_finished) = 0;
|
||||
virtual Status ProbeSingleBatch(size_t thread_index, ExecBatch batch) = 0;
|
||||
virtual Status ProbingFinished(size_t thread_index) = 0;
|
||||
virtual void Abort(TaskScheduler::AbortContinuationImpl pos_abort_callback) = 0;
|
||||
virtual std::string ToString() const = 0;
|
||||
|
||||
static Result<std::unique_ptr<HashJoinImpl>> MakeBasic();
|
||||
static Result<std::unique_ptr<HashJoinImpl>> MakeSwiss();
|
||||
|
||||
protected:
|
||||
arrow::util::tracing::Span span_;
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,318 @@
|
||||
// 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 <memory>
|
||||
#include <unordered_map>
|
||||
|
||||
#include "arrow/acero/schema_util.h"
|
||||
#include "arrow/compute/exec.h"
|
||||
#include "arrow/compute/row/row_encoder_internal.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/status.h"
|
||||
#include "arrow/type.h"
|
||||
|
||||
// This file contains hash join logic related to handling of dictionary encoded key
|
||||
// columns.
|
||||
//
|
||||
// A key column from probe side of the join can be matched against a key column from build
|
||||
// side of the join, as long as the underlying value types are equal. That means that:
|
||||
// - both scalars and arrays can be used and even mixed in the same column
|
||||
// - dictionary column can be matched against non-dictionary column if underlying value
|
||||
// types are equal
|
||||
// - dictionary column can be matched against dictionary column with a different index
|
||||
// type, and potentially using a different dictionary, if underlying value types are equal
|
||||
//
|
||||
// We currently require in hash join that for all dictionary encoded columns, the same
|
||||
// dictionary is used in all input exec batches.
|
||||
//
|
||||
// In order to allow matching columns with different dictionaries, different dictionary
|
||||
// index types, and dictionary key against non-dictionary key, internally comparisons will
|
||||
// be evaluated after remapping values on both sides of the join to a common
|
||||
// representation (which will be called "unified representation"). This common
|
||||
// representation is a column of int32() type (not a dictionary column). It represents an
|
||||
// index in the unified dictionary computed for the (only) dictionary present on build
|
||||
// side (an empty dictionary is still created for an empty build side). Null value is
|
||||
// always represented in this common representation as null int32 value, unified
|
||||
// dictionary will never contain a null value (so there is no ambiguity of representing
|
||||
// nulls as either index to a null entry in the dictionary or null index).
|
||||
//
|
||||
// Unified dictionary represents values present on build side. There may be values on
|
||||
// probe side that are not present in it. All such values, that are not null, are mapped
|
||||
// in the common representation to a special constant kMissingValueId.
|
||||
//
|
||||
|
||||
namespace arrow {
|
||||
|
||||
using compute::ExecBatch;
|
||||
using compute::ExecContext;
|
||||
using compute::internal::RowEncoder;
|
||||
|
||||
namespace acero {
|
||||
|
||||
/// Helper class with operations that are stateless and common to processing of dictionary
|
||||
/// keys on both build and probe side.
|
||||
class HashJoinDictUtil {
|
||||
public:
|
||||
// Null values in unified representation are always represented as null that has
|
||||
// corresponding integer set to this constant
|
||||
static constexpr int32_t kNullId = 0;
|
||||
// Constant representing a value, that is not null, missing on the build side, in
|
||||
// unified representation.
|
||||
static constexpr int32_t kMissingValueId = -1;
|
||||
|
||||
// Check if data types of corresponding pair of key column on build and probe side are
|
||||
// compatible
|
||||
static bool KeyDataTypesValid(const std::shared_ptr<DataType>& probe_data_type,
|
||||
const std::shared_ptr<DataType>& build_data_type);
|
||||
|
||||
// Input must be dictionary array or dictionary scalar.
|
||||
// A precomputed and provided here lookup table in the form of int32() array will be
|
||||
// used to remap input indices to unified representation.
|
||||
//
|
||||
static Result<std::shared_ptr<ArrayData>> IndexRemapUsingLUT(
|
||||
ExecContext* ctx, const Datum& indices, int64_t batch_length,
|
||||
const std::shared_ptr<ArrayData>& map_array,
|
||||
const std::shared_ptr<DataType>& data_type);
|
||||
|
||||
// Return int32() array that contains indices of input dictionary array or scalar after
|
||||
// type casting.
|
||||
static Result<std::shared_ptr<ArrayData>> ConvertToInt32(
|
||||
const std::shared_ptr<DataType>& from_type, const Datum& input,
|
||||
int64_t batch_length, ExecContext* ctx);
|
||||
|
||||
// Return an array that contains elements of input int32() array after casting to a
|
||||
// given integer type. This is used for mapping unified representation stored in the
|
||||
// hash table on build side back to original input data type of hash join, when
|
||||
// outputting hash join results to parent exec node.
|
||||
//
|
||||
static Result<std::shared_ptr<ArrayData>> ConvertFromInt32(
|
||||
const std::shared_ptr<DataType>& to_type, const Datum& input, int64_t batch_length,
|
||||
ExecContext* ctx);
|
||||
|
||||
// Return dictionary referenced in either dictionary array or dictionary scalar
|
||||
static std::shared_ptr<Array> ExtractDictionary(const Datum& data);
|
||||
};
|
||||
|
||||
/// Implements processing of dictionary arrays/scalars in key columns on the build side of
|
||||
/// a hash join.
|
||||
/// Each instance of this class corresponds to a single column and stores and
|
||||
/// processes only the information related to that column.
|
||||
/// Const methods are thread-safe, non-const methods are not (the caller must make sure
|
||||
/// that only one thread at any time will access them).
|
||||
///
|
||||
class HashJoinDictBuild {
|
||||
public:
|
||||
// Returns true if the key column (described in input by its data type) requires any
|
||||
// pre- or post-processing related to handling dictionaries.
|
||||
//
|
||||
static bool KeyNeedsProcessing(const std::shared_ptr<DataType>& build_data_type) {
|
||||
return (build_data_type->id() == Type::DICTIONARY);
|
||||
}
|
||||
|
||||
// Data type of unified representation
|
||||
static std::shared_ptr<DataType> DataTypeAfterRemapping() { return int32(); }
|
||||
|
||||
// Should be called only once in hash join, before processing any build or probe
|
||||
// batches.
|
||||
//
|
||||
// Takes a pointer to the dictionary for a corresponding key column on the build side as
|
||||
// an input. If the build side is empty, it still needs to be called, but with
|
||||
// dictionary pointer set to null.
|
||||
//
|
||||
// Currently it is required that all input batches on build side share the same
|
||||
// dictionary. For each input batch during its pre-processing, dictionary will be
|
||||
// checked and error will be returned if it is different then the one provided in the
|
||||
// call to this method.
|
||||
//
|
||||
// Unifies the dictionary. The order of the values is still preserved.
|
||||
// Null and duplicate entries are removed. If the dictionary is already unified, its
|
||||
// copy will be produced and stored within this class.
|
||||
//
|
||||
// Prepares the mapping from ids within original dictionary to the ids in the resulting
|
||||
// dictionary. This is used later on to pre-process (map to unified representation) key
|
||||
// column on build side.
|
||||
//
|
||||
// Prepares the reverse mapping (in the form of hash table) from values to the ids in
|
||||
// the resulting dictionary. This will be used later on to pre-process (map to unified
|
||||
// representation) key column on probe side. Values on probe side that are not present
|
||||
// in the original dictionary will be mapped to a special constant kMissingValueId. The
|
||||
// exception is made for nulls, which get always mapped to nulls (both when null is
|
||||
// represented as a dictionary id pointing to a null and a null dictionary id).
|
||||
//
|
||||
Status Init(ExecContext* ctx, std::shared_ptr<Array> dictionary,
|
||||
std::shared_ptr<DataType> index_type, std::shared_ptr<DataType> value_type);
|
||||
|
||||
// Remap array or scalar values into unified representation (array of int32()).
|
||||
// Outputs kMissingValueId if input value is not found in the unified dictionary.
|
||||
// Outputs null for null input value (with corresponding data set to kNullId).
|
||||
//
|
||||
Result<std::shared_ptr<ArrayData>> RemapInputValues(ExecContext* ctx,
|
||||
const Datum& values,
|
||||
int64_t batch_length) const;
|
||||
|
||||
// Remap dictionary array or dictionary scalar on build side to unified representation.
|
||||
// Dictionary referenced in the input must match the dictionary that was
|
||||
// given during initialization.
|
||||
// The output is a dictionary array that references unified dictionary.
|
||||
//
|
||||
Result<std::shared_ptr<ArrayData>> RemapInput(
|
||||
ExecContext* ctx, const Datum& indices, int64_t batch_length,
|
||||
const std::shared_ptr<DataType>& data_type) const;
|
||||
|
||||
// Outputs dictionary array referencing unified dictionary, given an array with 32-bit
|
||||
// ids.
|
||||
// Used to post-process values looked up in a hash table on build side of the hash join
|
||||
// before outputting to the parent exec node.
|
||||
//
|
||||
Result<std::shared_ptr<ArrayData>> RemapOutput(const ArrayData& indices32Bit,
|
||||
ExecContext* ctx) const;
|
||||
|
||||
// Release shared pointers and memory
|
||||
void CleanUp();
|
||||
|
||||
private:
|
||||
// Data type of dictionary ids for the input dictionary on build side
|
||||
std::shared_ptr<DataType> index_type_;
|
||||
// Data type of values for the input dictionary on build side
|
||||
std::shared_ptr<DataType> value_type_;
|
||||
// Mapping from (encoded as string) values to the ids in unified dictionary
|
||||
std::unordered_map<std::string, int32_t> hash_table_;
|
||||
// Mapping from input dictionary ids to unified dictionary ids
|
||||
std::shared_ptr<ArrayData> remapped_ids_;
|
||||
// Input dictionary
|
||||
std::shared_ptr<Array> dictionary_;
|
||||
// Unified dictionary
|
||||
std::shared_ptr<ArrayData> unified_dictionary_;
|
||||
};
|
||||
|
||||
/// Implements processing of dictionary arrays/scalars in key columns on the probe side of
|
||||
/// a hash join.
|
||||
/// Each instance of this class corresponds to a single column and stores and
|
||||
/// processes only the information related to that column.
|
||||
/// It is not thread-safe - every participating thread should use its own instance of
|
||||
/// this class.
|
||||
///
|
||||
class HashJoinDictProbe {
|
||||
public:
|
||||
static bool KeyNeedsProcessing(const std::shared_ptr<DataType>& probe_data_type,
|
||||
const std::shared_ptr<DataType>& build_data_type);
|
||||
|
||||
// Data type of the result of remapping input key column.
|
||||
//
|
||||
// The result of remapping is what is used in hash join for matching keys on build and
|
||||
// probe side. The exact data types may be different, as described below, and therefore
|
||||
// a common representation is needed for simplifying comparisons of pairs of keys on
|
||||
// both sides.
|
||||
//
|
||||
// We support matching key that is of non-dictionary type with key that is of dictionary
|
||||
// type, as long as the underlying value types are equal. We support matching when both
|
||||
// keys are of dictionary type, regardless whether underlying dictionary index types are
|
||||
// the same or not.
|
||||
//
|
||||
static std::shared_ptr<DataType> DataTypeAfterRemapping(
|
||||
const std::shared_ptr<DataType>& build_data_type);
|
||||
|
||||
// Should only be called if KeyNeedsProcessing method returns true for a pair of
|
||||
// corresponding key columns from build and probe side.
|
||||
// Converts values in order to match the common representation for
|
||||
// both build and probe side used in hash table comparison.
|
||||
// Supports arrays and scalars as input.
|
||||
// Argument opt_build_side should be null if dictionary key on probe side is matched
|
||||
// with non-dictionary key on build side.
|
||||
//
|
||||
Result<std::shared_ptr<ArrayData>> RemapInput(
|
||||
const HashJoinDictBuild* opt_build_side, const Datum& data, int64_t batch_length,
|
||||
const std::shared_ptr<DataType>& probe_data_type,
|
||||
const std::shared_ptr<DataType>& build_data_type, ExecContext* ctx);
|
||||
|
||||
void CleanUp();
|
||||
|
||||
private:
|
||||
// May be null if probe side key is non-dictionary. Otherwise it is used to verify that
|
||||
// only a single dictionary is referenced in exec batch on probe side of hash join.
|
||||
std::shared_ptr<Array> dictionary_;
|
||||
// Mapping from dictionary on probe side of hash join (if it is used) to unified
|
||||
// representation.
|
||||
std::shared_ptr<ArrayData> remapped_ids_;
|
||||
// Encoder of key columns that uses unified representation instead of original data type
|
||||
// for key columns that need to use it (have dictionaries on either side of the join).
|
||||
RowEncoder encoder_;
|
||||
};
|
||||
|
||||
// Encapsulates dictionary handling logic for build side of hash join.
|
||||
//
|
||||
class HashJoinDictBuildMulti {
|
||||
public:
|
||||
Status Init(const SchemaProjectionMaps<HashJoinProjection>& proj_map,
|
||||
const ExecBatch* opt_non_empty_batch, ExecContext* ctx);
|
||||
static void InitEncoder(const SchemaProjectionMaps<HashJoinProjection>& proj_map,
|
||||
RowEncoder* encoder, ExecContext* ctx);
|
||||
Status EncodeBatch(size_t thread_index,
|
||||
const SchemaProjectionMaps<HashJoinProjection>& proj_map,
|
||||
const ExecBatch& batch, RowEncoder* encoder, ExecContext* ctx) const;
|
||||
Status PostDecode(const SchemaProjectionMaps<HashJoinProjection>& proj_map,
|
||||
ExecBatch* decoded_key_batch, ExecContext* ctx);
|
||||
const HashJoinDictBuild& get_dict_build(int icol) const { return remap_imp_[icol]; }
|
||||
|
||||
private:
|
||||
std::vector<bool> needs_remap_;
|
||||
std::vector<HashJoinDictBuild> remap_imp_;
|
||||
};
|
||||
|
||||
// Encapsulates dictionary handling logic for probe side of hash join
|
||||
//
|
||||
class HashJoinDictProbeMulti {
|
||||
public:
|
||||
void Init(size_t num_threads);
|
||||
bool BatchRemapNeeded(size_t thread_index,
|
||||
const SchemaProjectionMaps<HashJoinProjection>& proj_map_probe,
|
||||
const SchemaProjectionMaps<HashJoinProjection>& proj_map_build,
|
||||
ExecContext* ctx);
|
||||
Status EncodeBatch(size_t thread_index,
|
||||
const SchemaProjectionMaps<HashJoinProjection>& proj_map_probe,
|
||||
const SchemaProjectionMaps<HashJoinProjection>& proj_map_build,
|
||||
const HashJoinDictBuildMulti& dict_build, const ExecBatch& batch,
|
||||
RowEncoder** out_encoder, ExecBatch* opt_out_key_batch,
|
||||
ExecContext* ctx);
|
||||
|
||||
private:
|
||||
void InitLocalStateIfNeeded(
|
||||
size_t thread_index, const SchemaProjectionMaps<HashJoinProjection>& proj_map_probe,
|
||||
const SchemaProjectionMaps<HashJoinProjection>& proj_map_build, ExecContext* ctx);
|
||||
static void InitEncoder(const SchemaProjectionMaps<HashJoinProjection>& proj_map_probe,
|
||||
const SchemaProjectionMaps<HashJoinProjection>& proj_map_build,
|
||||
RowEncoder* encoder, ExecContext* ctx);
|
||||
struct ThreadLocalState {
|
||||
bool is_initialized;
|
||||
// Whether any key column needs remapping (because of dictionaries used) before doing
|
||||
// join hash table lookups
|
||||
bool any_needs_remap;
|
||||
// Whether each key column needs remapping before doing join hash table lookups
|
||||
std::vector<bool> needs_remap;
|
||||
std::vector<HashJoinDictProbe> remap_imp;
|
||||
// Encoder of key columns that uses unified representation instead of original data
|
||||
// type for key columns that need to use it (have dictionaries on either side of the
|
||||
// join).
|
||||
RowEncoder post_remap_encoder;
|
||||
};
|
||||
std::vector<ThreadLocalState> local_states_;
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,103 @@
|
||||
// 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 <cassert>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/options.h"
|
||||
#include "arrow/acero/schema_util.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/status.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
using compute::ExecContext;
|
||||
|
||||
namespace acero {
|
||||
|
||||
class ARROW_ACERO_EXPORT HashJoinSchema {
|
||||
public:
|
||||
Status Init(JoinType join_type, const Schema& left_schema,
|
||||
const std::vector<FieldRef>& left_keys, const Schema& right_schema,
|
||||
const std::vector<FieldRef>& right_keys, const Expression& filter,
|
||||
const std::string& left_field_name_prefix,
|
||||
const std::string& right_field_name_prefix);
|
||||
|
||||
Status Init(JoinType join_type, const Schema& left_schema,
|
||||
const std::vector<FieldRef>& left_keys,
|
||||
const std::vector<FieldRef>& left_output, const Schema& right_schema,
|
||||
const std::vector<FieldRef>& right_keys,
|
||||
const std::vector<FieldRef>& right_output, const Expression& filter,
|
||||
const std::string& left_field_name_prefix,
|
||||
const std::string& right_field_name_prefix);
|
||||
|
||||
static Status ValidateSchemas(JoinType join_type, const Schema& left_schema,
|
||||
const std::vector<FieldRef>& left_keys,
|
||||
const std::vector<FieldRef>& left_output,
|
||||
const Schema& right_schema,
|
||||
const std::vector<FieldRef>& right_keys,
|
||||
const std::vector<FieldRef>& right_output,
|
||||
const std::string& left_field_name_prefix,
|
||||
const std::string& right_field_name_prefix);
|
||||
|
||||
bool HasDictionaries() const;
|
||||
|
||||
bool HasLargeBinary() const;
|
||||
|
||||
Result<Expression> BindFilter(Expression filter, const Schema& left_schema,
|
||||
const Schema& right_schema, ExecContext* exec_context);
|
||||
std::shared_ptr<Schema> MakeOutputSchema(const std::string& left_field_name_suffix,
|
||||
const std::string& right_field_name_suffix);
|
||||
|
||||
bool LeftPayloadIsEmpty() const { return PayloadIsEmpty(0); }
|
||||
|
||||
bool RightPayloadIsEmpty() const { return PayloadIsEmpty(1); }
|
||||
|
||||
static int kMissingField() {
|
||||
return SchemaProjectionMaps<HashJoinProjection>::kMissingField;
|
||||
}
|
||||
|
||||
SchemaProjectionMaps<HashJoinProjection> proj_maps[2];
|
||||
|
||||
private:
|
||||
static bool IsTypeSupported(const DataType& type);
|
||||
|
||||
Status CollectFilterColumns(std::vector<FieldRef>& left_filter,
|
||||
std::vector<FieldRef>& right_filter,
|
||||
const Expression& filter, const Schema& left_schema,
|
||||
const Schema& right_schema);
|
||||
|
||||
Expression RewriteFilterToUseFilterSchema(int right_filter_offset,
|
||||
const SchemaProjectionMap& left_to_filter,
|
||||
const SchemaProjectionMap& right_to_filter,
|
||||
const Expression& filter);
|
||||
|
||||
bool PayloadIsEmpty(int side) const {
|
||||
assert(side == 0 || side == 1);
|
||||
return proj_maps[side].num_cols(HashJoinProjection::PAYLOAD) == 0;
|
||||
}
|
||||
|
||||
static Result<std::vector<FieldRef>> ComputePayload(const Schema& schema,
|
||||
const std::vector<FieldRef>& output,
|
||||
const std::vector<FieldRef>& filter,
|
||||
const std::vector<FieldRef>& key);
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,81 @@
|
||||
// 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 <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/exec_plan.h"
|
||||
#include "arrow/acero/util.h"
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/compute/type_fwd.h"
|
||||
#include "arrow/status.h"
|
||||
#include "arrow/type_fwd.h"
|
||||
#include "arrow/util/cancel.h"
|
||||
#include "arrow/util/type_fwd.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
|
||||
/// A utility base class for simple exec nodes with one input
|
||||
///
|
||||
/// Pause/Resume Producing are forwarded appropriately
|
||||
/// There is nothing to do in StopProducingImpl
|
||||
///
|
||||
/// An AtomicCounter is used to keep track of when all data has arrived. When it
|
||||
/// has the Finish() method will be invoked
|
||||
class ARROW_ACERO_EXPORT MapNode : public ExecNode, public TracedNode {
|
||||
public:
|
||||
MapNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
|
||||
std::shared_ptr<Schema> output_schema);
|
||||
|
||||
Status InputFinished(ExecNode* input, int total_batches) override;
|
||||
|
||||
Status StartProducing() override;
|
||||
|
||||
void PauseProducing(ExecNode* output, int32_t counter) override;
|
||||
|
||||
void ResumeProducing(ExecNode* output, int32_t counter) override;
|
||||
|
||||
Status InputReceived(ExecNode* input, ExecBatch batch) override;
|
||||
|
||||
const Ordering& ordering() const override;
|
||||
|
||||
protected:
|
||||
Status StopProducingImpl() override;
|
||||
|
||||
/// Transform a batch
|
||||
///
|
||||
/// The output batch will have the same guarantee as the input batch
|
||||
/// If this was the last batch this call may trigger Finish()
|
||||
virtual Result<ExecBatch> ProcessBatch(ExecBatch batch) = 0;
|
||||
|
||||
/// Function called after all data has been received
|
||||
///
|
||||
/// By default this does nothing. Override this to provide a custom implementation.
|
||||
virtual void Finish();
|
||||
|
||||
protected:
|
||||
// Counter for the number of batches received
|
||||
AtomicCounter input_counter_;
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,874 @@
|
||||
// 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 <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/type_fwd.h"
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/compute/api_aggregate.h"
|
||||
#include "arrow/compute/api_vector.h"
|
||||
#include "arrow/compute/exec.h"
|
||||
#include "arrow/compute/expression.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/util/future.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
using compute::Aggregate;
|
||||
using compute::ExecBatch;
|
||||
using compute::Expression;
|
||||
using compute::literal;
|
||||
using compute::Ordering;
|
||||
using compute::SelectKOptions;
|
||||
using compute::SortOptions;
|
||||
|
||||
namespace internal {
|
||||
|
||||
class Executor;
|
||||
|
||||
} // namespace internal
|
||||
|
||||
namespace acero {
|
||||
|
||||
/// \brief This must not be used in release-mode
|
||||
struct DebugOptions;
|
||||
|
||||
using AsyncExecBatchGenerator = std::function<Future<std::optional<ExecBatch>>()>;
|
||||
|
||||
/// \addtogroup acero-nodes
|
||||
/// @{
|
||||
|
||||
/// \brief A base class for all options objects
|
||||
///
|
||||
/// The only time this is used directly is when a node has no configuration
|
||||
class ARROW_ACERO_EXPORT ExecNodeOptions {
|
||||
public:
|
||||
virtual ~ExecNodeOptions() = default;
|
||||
|
||||
/// \brief This must not be used in release-mode
|
||||
std::shared_ptr<DebugOptions> debug_opts;
|
||||
};
|
||||
|
||||
/// \brief A node representing a generic source of data for Acero
|
||||
///
|
||||
/// The source node will start calling `generator` during StartProducing. An initial
|
||||
/// task will be created that will call `generator`. It will not call `generator`
|
||||
/// reentrantly. If the source can be read in parallel then those details should be
|
||||
/// encapsulated within `generator`.
|
||||
///
|
||||
/// For each batch received a new task will be created to push that batch downstream.
|
||||
/// This task will slice smaller units of size `ExecPlan::kMaxBatchSize` from the
|
||||
/// parent batch and call InputReceived. Thus, if the `generator` yields a large
|
||||
/// batch it may result in several calls to InputReceived.
|
||||
///
|
||||
/// The SourceNode will, by default, assign an implicit ordering to outgoing batches.
|
||||
/// This is valid as long as the generator generates batches in a deterministic fashion.
|
||||
/// Currently, the only way to override this is to subclass the SourceNode.
|
||||
///
|
||||
/// This node is not generally used directly but can serve as the basis for various
|
||||
/// specialized nodes.
|
||||
class ARROW_ACERO_EXPORT SourceNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// Create an instance from values
|
||||
SourceNodeOptions(std::shared_ptr<Schema> output_schema,
|
||||
std::function<Future<std::optional<ExecBatch>>()> generator,
|
||||
Ordering ordering = Ordering::Unordered())
|
||||
: output_schema(std::move(output_schema)),
|
||||
generator(std::move(generator)),
|
||||
ordering(std::move(ordering)) {}
|
||||
|
||||
/// \brief the schema for batches that will be generated by this source
|
||||
std::shared_ptr<Schema> output_schema;
|
||||
/// \brief an asynchronous stream of batches ending with std::nullopt
|
||||
std::function<Future<std::optional<ExecBatch>>()> generator;
|
||||
/// \brief the order of the data, defaults to Ordering::Unordered
|
||||
Ordering ordering;
|
||||
};
|
||||
|
||||
/// \brief a node that generates data from a table already loaded in memory
|
||||
///
|
||||
/// The table source node will slice off chunks, defined by `max_batch_size`
|
||||
/// for parallel processing. The table source node extends source node and so these
|
||||
/// chunks will be iteratively processed in small batches. \see SourceNodeOptions
|
||||
/// for details.
|
||||
class ARROW_ACERO_EXPORT TableSourceNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
static constexpr int64_t kDefaultMaxBatchSize = 1 << 20;
|
||||
|
||||
/// Create an instance from values
|
||||
TableSourceNodeOptions(std::shared_ptr<Table> table,
|
||||
int64_t max_batch_size = kDefaultMaxBatchSize)
|
||||
: table(std::move(table)), max_batch_size(max_batch_size) {}
|
||||
|
||||
/// \brief a table which acts as the data source
|
||||
std::shared_ptr<Table> table;
|
||||
/// \brief size of batches to emit from this node
|
||||
/// If the table is larger the node will emit multiple batches from the
|
||||
/// the table to be processed in parallel.
|
||||
int64_t max_batch_size;
|
||||
};
|
||||
|
||||
/// \brief define a lazily resolved Arrow table.
|
||||
///
|
||||
/// The table uniquely identified by the names can typically be resolved at the time when
|
||||
/// the plan is to be consumed.
|
||||
///
|
||||
/// This node is for serialization purposes only and can never be executed.
|
||||
class ARROW_ACERO_EXPORT NamedTableNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// Create an instance from values
|
||||
NamedTableNodeOptions(std::vector<std::string> names, std::shared_ptr<Schema> schema)
|
||||
: names(std::move(names)), schema(std::move(schema)) {}
|
||||
|
||||
/// \brief the names to put in the serialized plan
|
||||
std::vector<std::string> names;
|
||||
/// \brief the output schema of the table
|
||||
std::shared_ptr<Schema> schema;
|
||||
};
|
||||
|
||||
/// \brief a source node which feeds data from a synchronous iterator of batches
|
||||
///
|
||||
/// ItMaker is a maker of an iterator of tabular data.
|
||||
///
|
||||
/// The node can be configured to use an I/O executor. If set then each time the
|
||||
/// iterator is polled a new I/O thread task will be created to do the polling. This
|
||||
/// allows a blocking iterator to stay off the CPU thread pool.
|
||||
template <typename ItMaker>
|
||||
class ARROW_ACERO_EXPORT SchemaSourceNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// Create an instance that will create a new task on io_executor for each iteration
|
||||
SchemaSourceNodeOptions(std::shared_ptr<Schema> schema, ItMaker it_maker,
|
||||
arrow::internal::Executor* io_executor)
|
||||
: schema(std::move(schema)),
|
||||
it_maker(std::move(it_maker)),
|
||||
io_executor(io_executor),
|
||||
requires_io(true) {}
|
||||
|
||||
/// Create an instance that will either iterate synchronously or use the default I/O
|
||||
/// executor
|
||||
SchemaSourceNodeOptions(std::shared_ptr<Schema> schema, ItMaker it_maker,
|
||||
bool requires_io = false)
|
||||
: schema(std::move(schema)),
|
||||
it_maker(std::move(it_maker)),
|
||||
io_executor(NULLPTR),
|
||||
requires_io(requires_io) {}
|
||||
|
||||
/// \brief The schema of the record batches from the iterator
|
||||
std::shared_ptr<Schema> schema;
|
||||
|
||||
/// \brief A maker of an iterator which acts as the data source
|
||||
ItMaker it_maker;
|
||||
|
||||
/// \brief The executor to use for scanning the iterator
|
||||
///
|
||||
/// Defaults to the default I/O executor. Only used if requires_io is true.
|
||||
/// If requires_io is false then this MUST be nullptr.
|
||||
arrow::internal::Executor* io_executor;
|
||||
|
||||
/// \brief If true then items will be fetched from the iterator on a dedicated I/O
|
||||
/// thread to keep I/O off the CPU thread
|
||||
bool requires_io;
|
||||
};
|
||||
|
||||
/// a source node that reads from a RecordBatchReader
|
||||
///
|
||||
/// Each iteration of the RecordBatchReader will be run on a new thread task created
|
||||
/// on the I/O thread pool.
|
||||
class ARROW_ACERO_EXPORT RecordBatchReaderSourceNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// Create an instance from values
|
||||
RecordBatchReaderSourceNodeOptions(std::shared_ptr<RecordBatchReader> reader,
|
||||
arrow::internal::Executor* io_executor = NULLPTR)
|
||||
: reader(std::move(reader)), io_executor(io_executor) {}
|
||||
|
||||
/// \brief The RecordBatchReader which acts as the data source
|
||||
std::shared_ptr<RecordBatchReader> reader;
|
||||
|
||||
/// \brief The executor to use for the reader
|
||||
///
|
||||
/// Defaults to the default I/O executor.
|
||||
arrow::internal::Executor* io_executor;
|
||||
};
|
||||
|
||||
/// a source node that reads from an iterator of array vectors
|
||||
using ArrayVectorIteratorMaker = std::function<Iterator<std::shared_ptr<ArrayVector>>()>;
|
||||
/// \brief An extended Source node which accepts a schema and array-vectors
|
||||
class ARROW_ACERO_EXPORT ArrayVectorSourceNodeOptions
|
||||
: public SchemaSourceNodeOptions<ArrayVectorIteratorMaker> {
|
||||
using SchemaSourceNodeOptions::SchemaSourceNodeOptions;
|
||||
};
|
||||
|
||||
/// a source node that reads from an iterator of ExecBatch
|
||||
using ExecBatchIteratorMaker = std::function<Iterator<std::shared_ptr<ExecBatch>>()>;
|
||||
/// \brief An extended Source node which accepts a schema and exec-batches
|
||||
class ARROW_ACERO_EXPORT ExecBatchSourceNodeOptions
|
||||
: public SchemaSourceNodeOptions<ExecBatchIteratorMaker> {
|
||||
public:
|
||||
using SchemaSourceNodeOptions::SchemaSourceNodeOptions;
|
||||
ExecBatchSourceNodeOptions(std::shared_ptr<Schema> schema,
|
||||
std::vector<ExecBatch> batches,
|
||||
::arrow::internal::Executor* io_executor);
|
||||
ExecBatchSourceNodeOptions(std::shared_ptr<Schema> schema,
|
||||
std::vector<ExecBatch> batches, bool requires_io = false);
|
||||
};
|
||||
|
||||
using RecordBatchIteratorMaker = std::function<Iterator<std::shared_ptr<RecordBatch>>()>;
|
||||
/// a source node that reads from an iterator of RecordBatch
|
||||
class ARROW_ACERO_EXPORT RecordBatchSourceNodeOptions
|
||||
: public SchemaSourceNodeOptions<RecordBatchIteratorMaker> {
|
||||
using SchemaSourceNodeOptions::SchemaSourceNodeOptions;
|
||||
};
|
||||
|
||||
/// \brief a node which excludes some rows from batches passed through it
|
||||
///
|
||||
/// filter_expression will be evaluated against each batch which is pushed to
|
||||
/// this node. Any rows for which filter_expression does not evaluate to `true` will be
|
||||
/// excluded in the batch emitted by this node.
|
||||
///
|
||||
/// This node will emit empty batches if all rows are excluded. This is done
|
||||
/// to avoid gaps in the ordering.
|
||||
class ARROW_ACERO_EXPORT FilterNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// \brief create an instance from values
|
||||
explicit FilterNodeOptions(Expression filter_expression)
|
||||
: filter_expression(std::move(filter_expression)) {}
|
||||
|
||||
/// \brief the expression to filter batches
|
||||
///
|
||||
/// The return type of this expression must be boolean
|
||||
Expression filter_expression;
|
||||
};
|
||||
|
||||
/// \brief a node which selects a specified subset from the input
|
||||
class ARROW_ACERO_EXPORT FetchNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
static constexpr std::string_view kName = "fetch";
|
||||
/// \brief create an instance from values
|
||||
FetchNodeOptions(int64_t offset, int64_t count) : offset(offset), count(count) {}
|
||||
/// \brief the number of rows to skip
|
||||
int64_t offset;
|
||||
/// \brief the number of rows to keep (not counting skipped rows)
|
||||
int64_t count;
|
||||
};
|
||||
|
||||
/// \brief a node which executes expressions on input batches, producing batches
|
||||
/// of the same length with new columns.
|
||||
///
|
||||
/// Each expression will be evaluated against each batch which is pushed to
|
||||
/// this node to produce a corresponding output column.
|
||||
///
|
||||
/// If names are not provided, the string representations of exprs will be used.
|
||||
class ARROW_ACERO_EXPORT ProjectNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// \brief create an instance from values
|
||||
explicit ProjectNodeOptions(std::vector<Expression> expressions,
|
||||
std::vector<std::string> names = {})
|
||||
: expressions(std::move(expressions)), names(std::move(names)) {}
|
||||
|
||||
/// \brief the expressions to run on the batches
|
||||
///
|
||||
/// The output will have one column for each expression. If you wish to keep any of
|
||||
/// the columns from the input then you should create a simple field_ref expression
|
||||
/// for that column.
|
||||
std::vector<Expression> expressions;
|
||||
/// \brief the names of the output columns
|
||||
///
|
||||
/// If this is not specified then the result of calling ToString on the expression will
|
||||
/// be used instead
|
||||
///
|
||||
/// This list should either be empty or have the same length as `expressions`
|
||||
std::vector<std::string> names;
|
||||
};
|
||||
|
||||
/// \brief a node which aggregates input batches and calculates summary statistics
|
||||
///
|
||||
/// The node can summarize the entire input or it can group the input with grouping keys
|
||||
/// and segment keys.
|
||||
///
|
||||
/// By default, the aggregate node is a pipeline breaker. It must accumulate all input
|
||||
/// before any output is produced. Segment keys are a performance optimization. If
|
||||
/// you know your input is already partitioned by one or more columns then you can
|
||||
/// specify these as segment keys. At each change in the segment keys the node will
|
||||
/// emit values for all data seen so far.
|
||||
///
|
||||
/// Segment keys are currently limited to single-threaded mode.
|
||||
///
|
||||
/// Both keys and segment-keys determine the group. However segment-keys are also used
|
||||
/// for determining grouping segments, which should be large, and allow streaming a
|
||||
/// partial aggregation result after processing each segment. One common use-case for
|
||||
/// segment-keys is ordered aggregation, in which the segment-key attribute specifies a
|
||||
/// column with non-decreasing values or a lexicographically-ordered set of such columns.
|
||||
///
|
||||
/// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
|
||||
/// expected to be a HashAggregate function. If the keys attribute is an empty vector,
|
||||
/// then each aggregate is assumed to be a ScalarAggregate function.
|
||||
///
|
||||
/// If the segment_keys attribute is a non-empty vector, then segmented aggregation, as
|
||||
/// described above, applies.
|
||||
///
|
||||
/// The keys and segment_keys vectors must be disjoint.
|
||||
///
|
||||
/// If no measures are provided then you will simply get the list of unique keys.
|
||||
///
|
||||
/// This node outputs segment keys first, followed by regular keys, followed by one
|
||||
/// column for each aggregate.
|
||||
class ARROW_ACERO_EXPORT AggregateNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// \brief create an instance from values
|
||||
explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
|
||||
std::vector<FieldRef> keys = {},
|
||||
std::vector<FieldRef> segment_keys = {})
|
||||
: aggregates(std::move(aggregates)),
|
||||
keys(std::move(keys)),
|
||||
segment_keys(std::move(segment_keys)) {}
|
||||
|
||||
// aggregations which will be applied to the targeted fields
|
||||
std::vector<Aggregate> aggregates;
|
||||
// keys by which aggregations will be grouped (optional)
|
||||
std::vector<FieldRef> keys;
|
||||
// keys by which aggregations will be segmented (optional)
|
||||
std::vector<FieldRef> segment_keys;
|
||||
};
|
||||
|
||||
/// \brief a default value at which backpressure will be applied
|
||||
constexpr int32_t kDefaultBackpressureHighBytes = 1 << 30; // 1GiB
|
||||
/// \brief a default value at which backpressure will be removed
|
||||
constexpr int32_t kDefaultBackpressureLowBytes = 1 << 28; // 256MiB
|
||||
|
||||
/// \brief an interface that can be queried for backpressure statistics
|
||||
class ARROW_ACERO_EXPORT BackpressureMonitor {
|
||||
public:
|
||||
virtual ~BackpressureMonitor() = default;
|
||||
/// \brief fetches the number of bytes currently queued up
|
||||
virtual uint64_t bytes_in_use() = 0;
|
||||
/// \brief checks to see if backpressure is currently applied
|
||||
virtual bool is_paused() = 0;
|
||||
};
|
||||
|
||||
/// \brief Options to control backpressure behavior
|
||||
struct ARROW_ACERO_EXPORT BackpressureOptions {
|
||||
/// \brief Create default options that perform no backpressure
|
||||
BackpressureOptions() : resume_if_below(0), pause_if_above(0) {}
|
||||
/// \brief Create options that will perform backpressure
|
||||
///
|
||||
/// \param resume_if_below The producer should resume producing if the backpressure
|
||||
/// queue has fewer than resume_if_below items.
|
||||
/// \param pause_if_above The producer should pause producing if the backpressure
|
||||
/// queue has more than pause_if_above items
|
||||
BackpressureOptions(uint64_t resume_if_below, uint64_t pause_if_above)
|
||||
: resume_if_below(resume_if_below), pause_if_above(pause_if_above) {}
|
||||
|
||||
/// \brief create an instance using default values for backpressure limits
|
||||
static BackpressureOptions DefaultBackpressure() {
|
||||
return BackpressureOptions(kDefaultBackpressureLowBytes,
|
||||
kDefaultBackpressureHighBytes);
|
||||
}
|
||||
|
||||
/// \brief helper method to determine if backpressure is disabled
|
||||
/// \return true if pause_if_above is greater than zero, false otherwise
|
||||
bool should_apply_backpressure() const { return pause_if_above > 0; }
|
||||
|
||||
/// \brief the number of bytes at which the producer should resume producing
|
||||
uint64_t resume_if_below;
|
||||
/// \brief the number of bytes at which the producer should pause producing
|
||||
///
|
||||
/// If this is <= 0 then backpressure will be disabled
|
||||
uint64_t pause_if_above;
|
||||
};
|
||||
|
||||
/// \brief a sink node which collects results in a queue
|
||||
///
|
||||
/// Emitted batches will only be ordered if there is a meaningful ordering
|
||||
/// and sequence_output is not set to false.
|
||||
class ARROW_ACERO_EXPORT SinkNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
explicit SinkNodeOptions(std::function<Future<std::optional<ExecBatch>>()>* generator,
|
||||
std::shared_ptr<Schema>* schema,
|
||||
BackpressureOptions backpressure = {},
|
||||
BackpressureMonitor** backpressure_monitor = NULLPTR,
|
||||
std::optional<bool> sequence_output = std::nullopt)
|
||||
: generator(generator),
|
||||
schema(schema),
|
||||
backpressure(backpressure),
|
||||
backpressure_monitor(backpressure_monitor),
|
||||
sequence_output(sequence_output) {}
|
||||
|
||||
explicit SinkNodeOptions(std::function<Future<std::optional<ExecBatch>>()>* generator,
|
||||
BackpressureOptions backpressure = {},
|
||||
BackpressureMonitor** backpressure_monitor = NULLPTR,
|
||||
std::optional<bool> sequence_output = std::nullopt)
|
||||
: generator(generator),
|
||||
schema(NULLPTR),
|
||||
backpressure(std::move(backpressure)),
|
||||
backpressure_monitor(backpressure_monitor),
|
||||
sequence_output(sequence_output) {}
|
||||
|
||||
/// \brief A pointer to a generator of batches.
|
||||
///
|
||||
/// This will be set when the node is added to the plan and should be used to consume
|
||||
/// data from the plan. If this function is not called frequently enough then the sink
|
||||
/// node will start to accumulate data and may apply backpressure.
|
||||
std::function<Future<std::optional<ExecBatch>>()>* generator;
|
||||
/// \brief A pointer which will be set to the schema of the generated batches
|
||||
///
|
||||
/// This is optional, if nullptr is passed in then it will be ignored.
|
||||
/// This will be set when the node is added to the plan, before StartProducing is called
|
||||
std::shared_ptr<Schema>* schema;
|
||||
/// \brief Options to control when to apply backpressure
|
||||
///
|
||||
/// This is optional, the default is to never apply backpressure. If the plan is not
|
||||
/// consumed quickly enough the system may eventually run out of memory.
|
||||
BackpressureOptions backpressure;
|
||||
/// \brief A pointer to a backpressure monitor
|
||||
///
|
||||
/// This will be set when the node is added to the plan. This can be used to inspect
|
||||
/// the amount of data currently queued in the sink node. This is an optional utility
|
||||
/// and backpressure can be applied even if this is not used.
|
||||
BackpressureMonitor** backpressure_monitor;
|
||||
/// \brief Controls whether batches should be emitted immediately or sequenced in order
|
||||
///
|
||||
/// \see QueryOptions for more details
|
||||
std::optional<bool> sequence_output;
|
||||
};
|
||||
|
||||
/// \brief Control used by a SinkNodeConsumer to pause & resume
|
||||
///
|
||||
/// Callers should ensure that they do not call Pause and Resume simultaneously and they
|
||||
/// should sequence things so that a call to Pause() is always followed by an eventual
|
||||
/// call to Resume()
|
||||
class ARROW_ACERO_EXPORT BackpressureControl {
|
||||
public:
|
||||
virtual ~BackpressureControl() = default;
|
||||
/// \brief Ask the input to pause
|
||||
///
|
||||
/// This is best effort, batches may continue to arrive
|
||||
/// Must eventually be followed by a call to Resume() or deadlock will occur
|
||||
virtual void Pause() = 0;
|
||||
/// \brief Ask the input to resume
|
||||
virtual void Resume() = 0;
|
||||
};
|
||||
|
||||
/// \brief a sink node that consumes the data as part of the plan using callbacks
|
||||
class ARROW_ACERO_EXPORT SinkNodeConsumer {
|
||||
public:
|
||||
virtual ~SinkNodeConsumer() = default;
|
||||
/// \brief Prepare any consumer state
|
||||
///
|
||||
/// This will be run once the schema is finalized as the plan is starting and
|
||||
/// before any calls to Consume. A common use is to save off the schema so that
|
||||
/// batches can be interpreted.
|
||||
virtual Status Init(const std::shared_ptr<Schema>& schema,
|
||||
BackpressureControl* backpressure_control, ExecPlan* plan) = 0;
|
||||
/// \brief Consume a batch of data
|
||||
virtual Status Consume(ExecBatch batch) = 0;
|
||||
/// \brief Signal to the consumer that the last batch has been delivered
|
||||
///
|
||||
/// The returned future should only finish when all outstanding tasks have completed
|
||||
///
|
||||
/// If the plan is ended early or aborts due to an error then this will not be
|
||||
/// called.
|
||||
virtual Future<> Finish() = 0;
|
||||
};
|
||||
|
||||
/// \brief Add a sink node which consumes data within the exec plan run
|
||||
class ARROW_ACERO_EXPORT ConsumingSinkNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
explicit ConsumingSinkNodeOptions(std::shared_ptr<SinkNodeConsumer> consumer,
|
||||
std::vector<std::string> names = {},
|
||||
std::optional<bool> sequence_output = std::nullopt)
|
||||
: consumer(std::move(consumer)),
|
||||
names(std::move(names)),
|
||||
sequence_output(sequence_output) {}
|
||||
|
||||
std::shared_ptr<SinkNodeConsumer> consumer;
|
||||
/// \brief Names to rename the sink's schema fields to
|
||||
///
|
||||
/// If specified then names must be provided for all fields. Currently, only a flat
|
||||
/// schema is supported (see GH-31875).
|
||||
///
|
||||
/// If not specified then names will be generated based on the source data.
|
||||
std::vector<std::string> names;
|
||||
/// \brief Controls whether batches should be emitted immediately or sequenced in order
|
||||
///
|
||||
/// \see QueryOptions for more details
|
||||
std::optional<bool> sequence_output;
|
||||
};
|
||||
|
||||
/// \brief Make a node which sorts rows passed through it
|
||||
///
|
||||
/// All batches pushed to this node will be accumulated, then sorted, by the given
|
||||
/// fields. Then sorted batches will be forwarded to the generator in sorted order.
|
||||
class ARROW_ACERO_EXPORT OrderBySinkNodeOptions : public SinkNodeOptions {
|
||||
public:
|
||||
/// \brief create an instance from values
|
||||
explicit OrderBySinkNodeOptions(
|
||||
SortOptions sort_options,
|
||||
std::function<Future<std::optional<ExecBatch>>()>* generator)
|
||||
: SinkNodeOptions(generator), sort_options(std::move(sort_options)) {}
|
||||
|
||||
/// \brief options describing which columns and direction to sort
|
||||
SortOptions sort_options;
|
||||
};
|
||||
|
||||
/// \brief Apply a new ordering to data
|
||||
///
|
||||
/// Currently this node works by accumulating all data, sorting, and then emitting
|
||||
/// the new data with an updated batch index.
|
||||
///
|
||||
/// Larger-than-memory sort is not currently supported.
|
||||
class ARROW_ACERO_EXPORT OrderByNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
static constexpr std::string_view kName = "order_by";
|
||||
explicit OrderByNodeOptions(Ordering ordering) : ordering(std::move(ordering)) {}
|
||||
|
||||
/// \brief The new ordering to apply to outgoing data
|
||||
Ordering ordering;
|
||||
};
|
||||
|
||||
enum class JoinType {
|
||||
LEFT_SEMI,
|
||||
RIGHT_SEMI,
|
||||
LEFT_ANTI,
|
||||
RIGHT_ANTI,
|
||||
INNER,
|
||||
LEFT_OUTER,
|
||||
RIGHT_OUTER,
|
||||
FULL_OUTER
|
||||
};
|
||||
|
||||
std::string ToString(JoinType t);
|
||||
|
||||
enum class JoinKeyCmp { EQ, IS };
|
||||
|
||||
/// \brief a node which implements a join operation using a hash table
|
||||
class ARROW_ACERO_EXPORT HashJoinNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
static constexpr const char* default_output_suffix_for_left = "";
|
||||
static constexpr const char* default_output_suffix_for_right = "";
|
||||
/// \brief create an instance from values that outputs all columns
|
||||
HashJoinNodeOptions(
|
||||
JoinType in_join_type, std::vector<FieldRef> in_left_keys,
|
||||
std::vector<FieldRef> in_right_keys, Expression filter = literal(true),
|
||||
std::string output_suffix_for_left = default_output_suffix_for_left,
|
||||
std::string output_suffix_for_right = default_output_suffix_for_right,
|
||||
bool disable_bloom_filter = false)
|
||||
: join_type(in_join_type),
|
||||
left_keys(std::move(in_left_keys)),
|
||||
right_keys(std::move(in_right_keys)),
|
||||
output_all(true),
|
||||
output_suffix_for_left(std::move(output_suffix_for_left)),
|
||||
output_suffix_for_right(std::move(output_suffix_for_right)),
|
||||
filter(std::move(filter)),
|
||||
disable_bloom_filter(disable_bloom_filter) {
|
||||
this->key_cmp.resize(this->left_keys.size());
|
||||
for (size_t i = 0; i < this->left_keys.size(); ++i) {
|
||||
this->key_cmp[i] = JoinKeyCmp::EQ;
|
||||
}
|
||||
}
|
||||
/// \brief create an instance from keys
|
||||
///
|
||||
/// This will create an inner join that outputs all columns and has no post join filter
|
||||
///
|
||||
/// `in_left_keys` should have the same length and types as `in_right_keys`
|
||||
/// @param in_left_keys the keys in the left input
|
||||
/// @param in_right_keys the keys in the right input
|
||||
HashJoinNodeOptions(std::vector<FieldRef> in_left_keys,
|
||||
std::vector<FieldRef> in_right_keys)
|
||||
: left_keys(std::move(in_left_keys)), right_keys(std::move(in_right_keys)) {
|
||||
this->join_type = JoinType::INNER;
|
||||
this->output_all = true;
|
||||
this->output_suffix_for_left = default_output_suffix_for_left;
|
||||
this->output_suffix_for_right = default_output_suffix_for_right;
|
||||
this->key_cmp.resize(this->left_keys.size());
|
||||
for (size_t i = 0; i < this->left_keys.size(); ++i) {
|
||||
this->key_cmp[i] = JoinKeyCmp::EQ;
|
||||
}
|
||||
this->filter = literal(true);
|
||||
}
|
||||
/// \brief create an instance from values using JoinKeyCmp::EQ for all comparisons
|
||||
HashJoinNodeOptions(
|
||||
JoinType join_type, std::vector<FieldRef> left_keys,
|
||||
std::vector<FieldRef> right_keys, std::vector<FieldRef> left_output,
|
||||
std::vector<FieldRef> right_output, Expression filter = literal(true),
|
||||
std::string output_suffix_for_left = default_output_suffix_for_left,
|
||||
std::string output_suffix_for_right = default_output_suffix_for_right,
|
||||
bool disable_bloom_filter = false)
|
||||
: join_type(join_type),
|
||||
left_keys(std::move(left_keys)),
|
||||
right_keys(std::move(right_keys)),
|
||||
output_all(false),
|
||||
left_output(std::move(left_output)),
|
||||
right_output(std::move(right_output)),
|
||||
output_suffix_for_left(std::move(output_suffix_for_left)),
|
||||
output_suffix_for_right(std::move(output_suffix_for_right)),
|
||||
filter(std::move(filter)),
|
||||
disable_bloom_filter(disable_bloom_filter) {
|
||||
this->key_cmp.resize(this->left_keys.size());
|
||||
for (size_t i = 0; i < this->left_keys.size(); ++i) {
|
||||
this->key_cmp[i] = JoinKeyCmp::EQ;
|
||||
}
|
||||
}
|
||||
/// \brief create an instance from values
|
||||
HashJoinNodeOptions(
|
||||
JoinType join_type, std::vector<FieldRef> left_keys,
|
||||
std::vector<FieldRef> right_keys, std::vector<FieldRef> left_output,
|
||||
std::vector<FieldRef> right_output, std::vector<JoinKeyCmp> key_cmp,
|
||||
Expression filter = literal(true),
|
||||
std::string output_suffix_for_left = default_output_suffix_for_left,
|
||||
std::string output_suffix_for_right = default_output_suffix_for_right,
|
||||
bool disable_bloom_filter = false)
|
||||
: join_type(join_type),
|
||||
left_keys(std::move(left_keys)),
|
||||
right_keys(std::move(right_keys)),
|
||||
output_all(false),
|
||||
left_output(std::move(left_output)),
|
||||
right_output(std::move(right_output)),
|
||||
key_cmp(std::move(key_cmp)),
|
||||
output_suffix_for_left(std::move(output_suffix_for_left)),
|
||||
output_suffix_for_right(std::move(output_suffix_for_right)),
|
||||
filter(std::move(filter)),
|
||||
disable_bloom_filter(disable_bloom_filter) {}
|
||||
|
||||
HashJoinNodeOptions() = default;
|
||||
|
||||
// type of join (inner, left, semi...)
|
||||
JoinType join_type = JoinType::INNER;
|
||||
// key fields from left input
|
||||
std::vector<FieldRef> left_keys;
|
||||
// key fields from right input
|
||||
std::vector<FieldRef> right_keys;
|
||||
// if set all valid fields from both left and right input will be output
|
||||
// (and field ref vectors for output fields will be ignored)
|
||||
bool output_all = false;
|
||||
// output fields passed from left input
|
||||
std::vector<FieldRef> left_output;
|
||||
// output fields passed from right input
|
||||
std::vector<FieldRef> right_output;
|
||||
// key comparison function (determines whether a null key is equal another null
|
||||
// key or not)
|
||||
std::vector<JoinKeyCmp> key_cmp;
|
||||
// suffix added to names of output fields coming from left input (used to distinguish,
|
||||
// if necessary, between fields of the same name in left and right input and can be left
|
||||
// empty if there are no name collisions)
|
||||
std::string output_suffix_for_left;
|
||||
// suffix added to names of output fields coming from right input
|
||||
std::string output_suffix_for_right;
|
||||
// residual filter which is applied to matching rows. Rows that do not match
|
||||
// the filter are not included. The filter is applied against the
|
||||
// concatenated input schema (left fields then right fields) and can reference
|
||||
// fields that are not included in the output.
|
||||
Expression filter = literal(true);
|
||||
// whether or not to disable Bloom filters in this join
|
||||
bool disable_bloom_filter = false;
|
||||
};
|
||||
|
||||
/// \brief a node which implements the asof join operation
|
||||
///
|
||||
/// 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
|
||||
/// together. Batches produced by each input must be ordered by the "on" key.
|
||||
/// This node will output one row for each row in the left table.
|
||||
class ARROW_ACERO_EXPORT AsofJoinNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// \brief Keys for one input table of the AsofJoin operation
|
||||
///
|
||||
/// The keys must be consistent across the input tables:
|
||||
/// Each "on" key must refer to a field of the same type and units across the tables.
|
||||
/// Each "by" key must refer to a list of fields of the same types across the tables.
|
||||
struct Keys {
|
||||
/// \brief "on" key for the join.
|
||||
///
|
||||
/// The input table must be sorted by the "on" key. Must be a single field of a common
|
||||
/// type. An inexact match is used on the "on" key, i.e. a row is considered a
|
||||
/// match if and only if `right.on - left.on` is in the range
|
||||
/// `[min(0, tolerance), max(0, tolerance)]`.
|
||||
/// Currently, the "on" key must be of an integer, date, or timestamp type.
|
||||
FieldRef on_key;
|
||||
/// \brief "by" key for the join.
|
||||
///
|
||||
/// Each input table must have each field of the "by" key. Exact equality is used for
|
||||
/// each field of the "by" key.
|
||||
/// Currently, each field of the "by" key must be of an integer, date, timestamp, or
|
||||
/// base-binary type.
|
||||
std::vector<FieldRef> by_key;
|
||||
};
|
||||
|
||||
AsofJoinNodeOptions(std::vector<Keys> input_keys, int64_t tolerance)
|
||||
: input_keys(std::move(input_keys)), tolerance(tolerance) {}
|
||||
|
||||
/// \brief AsofJoin keys per input table. At least two keys must be given. The first key
|
||||
/// corresponds to a left table and all other keys correspond to right tables for the
|
||||
/// as-of-join.
|
||||
///
|
||||
/// \see `Keys` for details.
|
||||
std::vector<Keys> input_keys;
|
||||
/// \brief Tolerance for inexact "on" key matching. A right row is considered a match
|
||||
/// with a left row if `right.on - left.on` is in the range
|
||||
/// `[min(0, tolerance), max(0, tolerance)]`. `tolerance` may be:
|
||||
/// - negative, in which case a past-as-of-join occurs (match iff
|
||||
/// `tolerance <= right.on - left.on <= 0`);
|
||||
/// - or positive, in which case a future-as-of-join occurs (match iff
|
||||
/// `0 <= right.on - left.on <= tolerance`);
|
||||
/// - or zero, in which case an exact-as-of-join occurs (match iff
|
||||
/// `right.on == left.on`).
|
||||
///
|
||||
/// The tolerance is interpreted in the same units as the "on" key.
|
||||
int64_t tolerance;
|
||||
};
|
||||
|
||||
/// \brief a node which select top_k/bottom_k rows passed through it
|
||||
///
|
||||
/// All batches pushed to this node will be accumulated, then selected, by the given
|
||||
/// fields. Then sorted batches will be forwarded to the generator in sorted order.
|
||||
class ARROW_ACERO_EXPORT SelectKSinkNodeOptions : public SinkNodeOptions {
|
||||
public:
|
||||
explicit SelectKSinkNodeOptions(
|
||||
SelectKOptions select_k_options,
|
||||
std::function<Future<std::optional<ExecBatch>>()>* generator)
|
||||
: SinkNodeOptions(generator), select_k_options(std::move(select_k_options)) {}
|
||||
|
||||
/// SelectK options
|
||||
SelectKOptions select_k_options;
|
||||
};
|
||||
|
||||
/// \brief a sink node which accumulates all output into a table
|
||||
class ARROW_ACERO_EXPORT TableSinkNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
/// \brief create an instance from values
|
||||
explicit TableSinkNodeOptions(std::shared_ptr<Table>* output_table,
|
||||
std::optional<bool> sequence_output = std::nullopt)
|
||||
: output_table(output_table), sequence_output(sequence_output) {}
|
||||
|
||||
/// \brief an "out parameter" specifying the table that will be created
|
||||
///
|
||||
/// Must not be null and remain valid for the entirety of the plan execution. After the
|
||||
/// plan has completed this will be set to point to the result table
|
||||
std::shared_ptr<Table>* output_table;
|
||||
/// \brief Controls whether batches should be emitted immediately or sequenced in order
|
||||
///
|
||||
/// \see QueryOptions for more details
|
||||
std::optional<bool> sequence_output;
|
||||
/// \brief Custom names to use for the columns.
|
||||
///
|
||||
/// If specified then names must be provided for all fields. Currently, only a flat
|
||||
/// schema is supported (see GH-31875).
|
||||
///
|
||||
/// If not specified then names will be generated based on the source data.
|
||||
std::vector<std::string> names;
|
||||
};
|
||||
|
||||
/// \brief a row template that describes one row that will be generated for each input row
|
||||
struct ARROW_ACERO_EXPORT PivotLongerRowTemplate {
|
||||
PivotLongerRowTemplate(std::vector<std::string> feature_values,
|
||||
std::vector<std::optional<FieldRef>> measurement_values)
|
||||
: feature_values(std::move(feature_values)),
|
||||
measurement_values(std::move(measurement_values)) {}
|
||||
/// A (typically unique) set of feature values for the template, usually derived from a
|
||||
/// column name
|
||||
///
|
||||
/// These will be used to populate the feature columns
|
||||
std::vector<std::string> feature_values;
|
||||
/// The fields containing the measurements to use for this row
|
||||
///
|
||||
/// These will be used to populate the measurement columns. If nullopt then nulls
|
||||
/// will be inserted for the given value.
|
||||
std::vector<std::optional<FieldRef>> measurement_values;
|
||||
};
|
||||
|
||||
/// \brief Reshape a table by turning some columns into additional rows
|
||||
///
|
||||
/// This operation is sometimes also referred to as UNPIVOT
|
||||
///
|
||||
/// This is typically done when there are multiple observations in each row in order to
|
||||
/// transform to a table containing a single observation per row.
|
||||
///
|
||||
/// For example:
|
||||
///
|
||||
/// | time | left_temp | right_temp |
|
||||
/// | ---- | --------- | ---------- |
|
||||
/// | 1 | 10 | 20 |
|
||||
/// | 2 | 15 | 18 |
|
||||
///
|
||||
/// The above table contains two observations per row. There is an implicit feature
|
||||
/// "location" (left vs right) and a measurement "temp". What we really want is:
|
||||
///
|
||||
/// | time | location | temp |
|
||||
/// | --- | --- | --- |
|
||||
/// | 1 | left | 10 |
|
||||
/// | 1 | right | 20 |
|
||||
/// | 2 | left | 15 |
|
||||
/// | 2 | right | 18 |
|
||||
///
|
||||
/// For a more complex example consider:
|
||||
///
|
||||
/// | time | ax1 | ay1 | bx1 | ay2 |
|
||||
/// | ---- | --- | --- | --- | --- |
|
||||
/// | 0 | 1 | 2 | 3 | 4 |
|
||||
///
|
||||
/// We can pretend a vs b and x vs y are features while 1 and 2 are two different
|
||||
/// kinds of measurements. We thus want to pivot to
|
||||
///
|
||||
/// | time | a/b | x/y | f1 | f2 |
|
||||
/// | ---- | --- | --- | ---- | ---- |
|
||||
/// | 0 | a | x | 1 | null |
|
||||
/// | 0 | a | y | 2 | 4 |
|
||||
/// | 0 | b | x | 3 | null |
|
||||
///
|
||||
/// To do this we create a row template for each combination of features. One should
|
||||
/// be able to do this purely by looking at the column names. For example, given the
|
||||
/// above columns "ax1", "ay1", "bx1", and "ay2" we know we have three feature
|
||||
/// combinations (a, x), (a, y), and (b, x). Similarly, we know we have two possible
|
||||
/// measurements, "1" and "2".
|
||||
///
|
||||
/// For each combination of features we create a row template. In each row template we
|
||||
/// describe the combination and then list which columns to use for the measurements.
|
||||
/// If a measurement doesn't exist for a given combination then we use nullopt.
|
||||
///
|
||||
/// So, for our above example, we have:
|
||||
///
|
||||
/// (a, x): names={"a", "x"}, values={"ax1", nullopt}
|
||||
/// (a, y): names={"a", "y"}, values={"ay1", "ay2"}
|
||||
/// (b, x): names={"b", "x"}, values={"bx1", nullopt}
|
||||
///
|
||||
/// Finishing it off we name our new columns:
|
||||
/// feature_field_names={"a/b","x/y"}
|
||||
/// measurement_field_names={"f1", "f2"}
|
||||
class ARROW_ACERO_EXPORT PivotLongerNodeOptions : public ExecNodeOptions {
|
||||
public:
|
||||
static constexpr std::string_view kName = "pivot_longer";
|
||||
/// One or more row templates to create new output rows
|
||||
///
|
||||
/// Normally there are at least two row templates. The output # of rows
|
||||
/// will be the input # of rows * the number of row templates
|
||||
std::vector<PivotLongerRowTemplate> row_templates;
|
||||
/// The names of the columns which describe the new features
|
||||
std::vector<std::string> feature_field_names;
|
||||
/// The names of the columns which represent the measurements
|
||||
std::vector<std::string> measurement_field_names;
|
||||
};
|
||||
|
||||
/// @}
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,56 @@
|
||||
// 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 <functional>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/options.h"
|
||||
#include "arrow/record_batch.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/status.h"
|
||||
#include "arrow/type.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
using compute::ExecContext;
|
||||
|
||||
namespace acero {
|
||||
|
||||
class OrderByImpl {
|
||||
public:
|
||||
virtual ~OrderByImpl() = default;
|
||||
|
||||
virtual void InputReceived(const std::shared_ptr<RecordBatch>& batch) = 0;
|
||||
|
||||
virtual Result<Datum> DoFinish() = 0;
|
||||
|
||||
virtual std::string ToString() const = 0;
|
||||
|
||||
static Result<std::unique_ptr<OrderByImpl>> MakeSort(
|
||||
ExecContext* ctx, const std::shared_ptr<Schema>& output_schema,
|
||||
const SortOptions& options);
|
||||
|
||||
static Result<std::unique_ptr<OrderByImpl>> MakeSelectK(
|
||||
ExecContext* ctx, const std::shared_ptr<Schema>& output_schema,
|
||||
const SelectKOptions& options);
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,186 @@
|
||||
// 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 <atomic>
|
||||
#include <cassert>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <random>
|
||||
|
||||
#include "arrow/acero/util.h"
|
||||
#include "arrow/buffer.h"
|
||||
#include "arrow/util/pcg_random.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
|
||||
class PartitionSort {
|
||||
public:
|
||||
/// \brief Bucket sort rows on partition ids in O(num_rows) time.
|
||||
///
|
||||
/// Include in the output exclusive cumulative sum of bucket sizes.
|
||||
/// This corresponds to ranges in the sorted array containing all row ids for
|
||||
/// each of the partitions.
|
||||
///
|
||||
/// prtn_ranges must be initialized and have at least num_prtns + 1 elements
|
||||
/// when this method returns prtn_ranges[i] will contains the total number of
|
||||
/// elements in partitions 0 through i. prtn_ranges[0] will be 0.
|
||||
///
|
||||
/// prtn_id_impl must be a function that takes in a row id (int) and returns
|
||||
/// a partition id (int). The returned partition id must be between 0 and
|
||||
/// num_prtns (exclusive).
|
||||
///
|
||||
/// output_pos_impl is a function that takes in a row id (int) and a position (int)
|
||||
/// in the bucket sorted output. The function should insert the row in the
|
||||
/// output.
|
||||
///
|
||||
/// For example:
|
||||
///
|
||||
/// in_arr: [5, 7, 2, 3, 5, 4]
|
||||
/// num_prtns: 3
|
||||
/// prtn_id_impl: [&in_arr] (int row_id) { return in_arr[row_id] / 3; }
|
||||
/// output_pos_impl: [&sorted_row_ids] (int row_id, int pos) { sorted_row_ids[pos] =
|
||||
/// row_id; }
|
||||
///
|
||||
/// After Execution
|
||||
/// sorted_row_ids: [2, 0, 3, 4, 5, 1]
|
||||
/// prtn_ranges: [0, 1, 5, 6]
|
||||
template <class INPUT_PRTN_ID_FN, class OUTPUT_POS_FN>
|
||||
static void Eval(int64_t num_rows, int num_prtns, uint16_t* prtn_ranges,
|
||||
INPUT_PRTN_ID_FN prtn_id_impl, OUTPUT_POS_FN output_pos_impl) {
|
||||
ARROW_DCHECK(num_rows > 0 && num_rows <= (1 << 15));
|
||||
ARROW_DCHECK(num_prtns >= 1 && num_prtns <= (1 << 15));
|
||||
|
||||
memset(prtn_ranges, 0, (num_prtns + 1) * sizeof(uint16_t));
|
||||
|
||||
for (int64_t i = 0; i < num_rows; ++i) {
|
||||
int prtn_id = static_cast<int>(prtn_id_impl(i));
|
||||
++prtn_ranges[prtn_id + 1];
|
||||
}
|
||||
|
||||
uint16_t sum = 0;
|
||||
for (int i = 0; i < num_prtns; ++i) {
|
||||
uint16_t sum_next = sum + prtn_ranges[i + 1];
|
||||
prtn_ranges[i + 1] = sum;
|
||||
sum = sum_next;
|
||||
}
|
||||
|
||||
for (int64_t i = 0; i < num_rows; ++i) {
|
||||
int prtn_id = static_cast<int>(prtn_id_impl(i));
|
||||
int pos = prtn_ranges[prtn_id + 1]++;
|
||||
output_pos_impl(i, pos);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/// \brief A control for synchronizing threads on a partitionable workload
|
||||
class PartitionLocks {
|
||||
public:
|
||||
PartitionLocks();
|
||||
~PartitionLocks();
|
||||
/// \brief Initializes the control, must be called before use
|
||||
///
|
||||
/// \param num_threads Maximum number of threads that will access the partitions
|
||||
/// \param num_prtns Number of partitions to synchronize
|
||||
void Init(size_t num_threads, int num_prtns);
|
||||
/// \brief Cleans up the control, it should not be used after this call
|
||||
void CleanUp();
|
||||
/// \brief Acquire a partition to work on one
|
||||
///
|
||||
/// \param thread_id The index of the thread trying to acquire the partition lock
|
||||
/// \param num_prtns Length of prtns_to_try, must be <= num_prtns used in Init
|
||||
/// \param prtns_to_try An array of partitions that still have remaining work
|
||||
/// \param limit_retries If false, this method will spinwait forever until success
|
||||
/// \param max_retries Max times to attempt checking out work before returning false
|
||||
/// \param[out] locked_prtn_id The id of the partition locked
|
||||
/// \param[out] locked_prtn_id_pos The index of the partition locked in prtns_to_try
|
||||
/// \return True if a partition was locked, false if max_retries was attempted
|
||||
/// without successfully acquiring a lock
|
||||
///
|
||||
/// This method is thread safe
|
||||
bool AcquirePartitionLock(size_t thread_id, int num_prtns, const int* prtns_to_try,
|
||||
bool limit_retries, int max_retries, int* locked_prtn_id,
|
||||
int* locked_prtn_id_pos);
|
||||
/// \brief Release a partition so that other threads can work on it
|
||||
void ReleasePartitionLock(int prtn_id);
|
||||
|
||||
// Executes (synchronously and using current thread) the same operation on a set of
|
||||
// multiple partitions. Tries to minimize partition locking overhead by randomizing and
|
||||
// adjusting order in which partitions are processed.
|
||||
//
|
||||
// PROCESS_PRTN_FN is a callback which will be executed for each partition after
|
||||
// acquiring the lock for that partition. It gets partition id as an argument.
|
||||
// IS_PRTN_EMPTY_FN is a callback which filters out (when returning true) partitions
|
||||
// with specific ids from processing.
|
||||
//
|
||||
template <typename IS_PRTN_EMPTY_FN, typename PROCESS_PRTN_FN>
|
||||
Status ForEachPartition(size_t thread_id,
|
||||
/*scratch space buffer with space for one element per partition;
|
||||
dirty in and dirty out*/
|
||||
int* temp_unprocessed_prtns, IS_PRTN_EMPTY_FN is_prtn_empty_fn,
|
||||
PROCESS_PRTN_FN process_prtn_fn) {
|
||||
int num_unprocessed_partitions = 0;
|
||||
for (int i = 0; i < num_prtns_; ++i) {
|
||||
bool is_prtn_empty = is_prtn_empty_fn(i);
|
||||
if (!is_prtn_empty) {
|
||||
temp_unprocessed_prtns[num_unprocessed_partitions++] = i;
|
||||
}
|
||||
}
|
||||
while (num_unprocessed_partitions > 0) {
|
||||
int locked_prtn_id;
|
||||
int locked_prtn_id_pos;
|
||||
AcquirePartitionLock(thread_id, num_unprocessed_partitions, temp_unprocessed_prtns,
|
||||
/*limit_retries=*/false, /*max_retries=*/-1, &locked_prtn_id,
|
||||
&locked_prtn_id_pos);
|
||||
{
|
||||
class AutoReleaseLock {
|
||||
public:
|
||||
AutoReleaseLock(PartitionLocks* locks, int prtn_id)
|
||||
: locks(locks), prtn_id(prtn_id) {}
|
||||
~AutoReleaseLock() { locks->ReleasePartitionLock(prtn_id); }
|
||||
PartitionLocks* locks;
|
||||
int prtn_id;
|
||||
} auto_release_lock(this, locked_prtn_id);
|
||||
ARROW_RETURN_NOT_OK(process_prtn_fn(locked_prtn_id));
|
||||
}
|
||||
if (locked_prtn_id_pos < num_unprocessed_partitions - 1) {
|
||||
temp_unprocessed_prtns[locked_prtn_id_pos] =
|
||||
temp_unprocessed_prtns[num_unprocessed_partitions - 1];
|
||||
}
|
||||
--num_unprocessed_partitions;
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
private:
|
||||
std::atomic<bool>* lock_ptr(int prtn_id);
|
||||
int random_int(size_t thread_id, int num_values);
|
||||
|
||||
struct PartitionLock {
|
||||
static constexpr int kCacheLineBytes = 64;
|
||||
std::atomic<bool> lock;
|
||||
uint8_t padding[kCacheLineBytes];
|
||||
};
|
||||
int num_prtns_;
|
||||
std::unique_ptr<PartitionLock[]> locks_;
|
||||
std::unique_ptr<arrow::random::pcg32_fast[]> rngs_;
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,151 @@
|
||||
// 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 <string_view>
|
||||
|
||||
#include "arrow/acero/exec_plan.h"
|
||||
#include "arrow/acero/task_util.h"
|
||||
#include "arrow/acero/util.h"
|
||||
#include "arrow/compute/exec.h"
|
||||
#include "arrow/io/interfaces.h"
|
||||
#include "arrow/util/async_util.h"
|
||||
#include "arrow/util/type_fwd.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
using compute::default_exec_context;
|
||||
using io::IOContext;
|
||||
|
||||
namespace acero {
|
||||
|
||||
class ARROW_ACERO_EXPORT QueryContext {
|
||||
public:
|
||||
QueryContext(QueryOptions opts = {},
|
||||
ExecContext exec_context = *default_exec_context());
|
||||
|
||||
Status Init(arrow::util::AsyncTaskScheduler* scheduler);
|
||||
|
||||
const ::arrow::internal::CpuInfo* cpu_info() const;
|
||||
int64_t hardware_flags() const;
|
||||
const QueryOptions& options() const { return options_; }
|
||||
MemoryPool* memory_pool() const { return exec_context_.memory_pool(); }
|
||||
::arrow::internal::Executor* executor() const { return exec_context_.executor(); }
|
||||
ExecContext* exec_context() { return &exec_context_; }
|
||||
IOContext* io_context() { return &io_context_; }
|
||||
TaskScheduler* scheduler() { return task_scheduler_.get(); }
|
||||
arrow::util::AsyncTaskScheduler* async_scheduler() { return async_scheduler_; }
|
||||
|
||||
size_t GetThreadIndex();
|
||||
size_t max_concurrency() const;
|
||||
|
||||
/// \brief Start an external task
|
||||
///
|
||||
/// This should be avoided if possible. It is kept in for now for legacy
|
||||
/// purposes. This should be called before the external task is started. If
|
||||
/// a valid future is returned then it should be marked complete when the
|
||||
/// external task has finished.
|
||||
///
|
||||
/// \param name A name to give the task for traceability and debugging
|
||||
///
|
||||
/// \return an invalid future if the plan has already ended, otherwise this
|
||||
/// returns a future that must be completed when the external task
|
||||
/// finishes.
|
||||
Result<Future<>> BeginExternalTask(std::string_view name);
|
||||
|
||||
/// \brief Add a single function as a task to the query's task group
|
||||
/// on the compute threadpool.
|
||||
///
|
||||
/// \param fn The task to run. Takes no arguments and returns a Status.
|
||||
/// \param name A name to give the task for traceability and debugging
|
||||
void ScheduleTask(std::function<Status()> fn, std::string_view name);
|
||||
/// \brief Add a single function as a task to the query's task group
|
||||
/// on the compute threadpool.
|
||||
///
|
||||
/// \param fn The task to run. Takes the thread index and returns a Status.
|
||||
/// \param name A name to give the task for traceability and debugging
|
||||
void ScheduleTask(std::function<Status(size_t)> fn, std::string_view name);
|
||||
/// \brief Add a single function as a task to the query's task group on
|
||||
/// the IO thread pool
|
||||
///
|
||||
/// \param fn The task to run. Returns a status.
|
||||
/// \param name A name to give the task for traceability and debugging
|
||||
void ScheduleIOTask(std::function<Status()> fn, std::string_view name);
|
||||
|
||||
// Register/Start TaskGroup is a way of performing a "Parallel For" pattern:
|
||||
// - The task function takes the thread index and the index of the task
|
||||
// - The on_finished function takes the thread index
|
||||
// Returns an integer ID that will be used to reference the task group in
|
||||
// StartTaskGroup. At runtime, call StartTaskGroup with the ID and the number of times
|
||||
// you'd like the task to be executed. The need to register a task group before use will
|
||||
// be removed after we rewrite the scheduler.
|
||||
/// \brief Register a "parallel for" task group with the scheduler
|
||||
///
|
||||
/// \param task The function implementing the task. Takes the thread_index and
|
||||
/// the task index.
|
||||
/// \param on_finished The function that gets run once all tasks have been completed.
|
||||
/// Takes the thread_index.
|
||||
///
|
||||
/// Must be called inside of ExecNode::Init.
|
||||
int RegisterTaskGroup(std::function<Status(size_t, int64_t)> task,
|
||||
std::function<Status(size_t)> on_finished);
|
||||
|
||||
/// \brief Start the task group with the specified ID. This can only
|
||||
/// be called once per task_group_id.
|
||||
///
|
||||
/// \param task_group_id The ID of the task group to run
|
||||
/// \param num_tasks The number of times to run the task
|
||||
Status StartTaskGroup(int task_group_id, int64_t num_tasks);
|
||||
|
||||
// This is an RAII class for keeping track of in-flight file IO. Useful for getting
|
||||
// an estimate of memory use, and how much memory we expect to be freed soon.
|
||||
// Returned by ReportTempFileIO.
|
||||
struct [[nodiscard]] TempFileIOMark {
|
||||
QueryContext* ctx_;
|
||||
size_t bytes_;
|
||||
|
||||
TempFileIOMark(QueryContext* ctx, size_t bytes) : ctx_(ctx), bytes_(bytes) {
|
||||
ctx_->in_flight_bytes_to_disk_.fetch_add(bytes_, std::memory_order_acquire);
|
||||
}
|
||||
|
||||
ARROW_DISALLOW_COPY_AND_ASSIGN(TempFileIOMark);
|
||||
|
||||
~TempFileIOMark() {
|
||||
ctx_->in_flight_bytes_to_disk_.fetch_sub(bytes_, std::memory_order_release);
|
||||
}
|
||||
};
|
||||
|
||||
TempFileIOMark ReportTempFileIO(size_t bytes) { return {this, bytes}; }
|
||||
|
||||
size_t GetCurrentTempFileIO() { return in_flight_bytes_to_disk_.load(); }
|
||||
|
||||
private:
|
||||
QueryOptions options_;
|
||||
// To be replaced with Acero-specific context once scheduler is done and
|
||||
// we don't need ExecContext for kernels
|
||||
ExecContext exec_context_;
|
||||
IOContext io_context_;
|
||||
|
||||
arrow::util::AsyncTaskScheduler* async_scheduler_ = NULLPTR;
|
||||
std::unique_ptr<TaskScheduler> task_scheduler_ = TaskScheduler::Make();
|
||||
|
||||
ThreadIndexer thread_indexer_;
|
||||
|
||||
std::atomic<size_t> in_flight_bytes_to_disk_{0};
|
||||
};
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,226 @@
|
||||
// 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 <cassert>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/type.h" // for DataType, FieldRef, Field and Schema
|
||||
|
||||
namespace arrow {
|
||||
|
||||
using internal::checked_cast;
|
||||
|
||||
namespace acero {
|
||||
|
||||
// Identifiers for all different row schemas that are used in a join
|
||||
//
|
||||
enum class HashJoinProjection : int {
|
||||
INPUT = 0,
|
||||
KEY = 1,
|
||||
PAYLOAD = 2,
|
||||
FILTER = 3,
|
||||
OUTPUT = 4
|
||||
};
|
||||
|
||||
struct SchemaProjectionMap {
|
||||
static constexpr int kMissingField = -1;
|
||||
int num_cols;
|
||||
const int* source_to_base;
|
||||
const int* base_to_target;
|
||||
inline int get(int i) const {
|
||||
assert(i >= 0 && i < num_cols);
|
||||
assert(source_to_base[i] != kMissingField);
|
||||
return base_to_target[source_to_base[i]];
|
||||
}
|
||||
};
|
||||
|
||||
/// Helper class for managing different projections of the same row schema.
|
||||
/// Used to efficiently map any field in one projection to a corresponding field in
|
||||
/// another projection.
|
||||
/// Materialized mappings are generated lazily at the time of the first access.
|
||||
/// Thread-safe apart from initialization.
|
||||
template <typename ProjectionIdEnum>
|
||||
class SchemaProjectionMaps {
|
||||
public:
|
||||
static constexpr int kMissingField = -1;
|
||||
|
||||
Status Init(ProjectionIdEnum full_schema_handle, const Schema& schema,
|
||||
const std::vector<ProjectionIdEnum>& projection_handles,
|
||||
const std::vector<const std::vector<FieldRef>*>& projections) {
|
||||
assert(projection_handles.size() == projections.size());
|
||||
ARROW_RETURN_NOT_OK(RegisterSchema(full_schema_handle, schema));
|
||||
for (size_t i = 0; i < projections.size(); ++i) {
|
||||
ARROW_RETURN_NOT_OK(
|
||||
RegisterProjectedSchema(projection_handles[i], *(projections[i]), schema));
|
||||
}
|
||||
RegisterEnd();
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
int num_cols(ProjectionIdEnum schema_handle) const {
|
||||
int id = schema_id(schema_handle);
|
||||
return static_cast<int>(schemas_[id].second.data_types.size());
|
||||
}
|
||||
|
||||
bool is_empty(ProjectionIdEnum schema_handle) const {
|
||||
return num_cols(schema_handle) == 0;
|
||||
}
|
||||
|
||||
const std::string& field_name(ProjectionIdEnum schema_handle, int field_id) const {
|
||||
int id = schema_id(schema_handle);
|
||||
return schemas_[id].second.field_names[field_id];
|
||||
}
|
||||
|
||||
const std::shared_ptr<DataType>& data_type(ProjectionIdEnum schema_handle,
|
||||
int field_id) const {
|
||||
int id = schema_id(schema_handle);
|
||||
return schemas_[id].second.data_types[field_id];
|
||||
}
|
||||
|
||||
const std::vector<std::shared_ptr<DataType>>& data_types(
|
||||
ProjectionIdEnum schema_handle) const {
|
||||
int id = schema_id(schema_handle);
|
||||
return schemas_[id].second.data_types;
|
||||
}
|
||||
|
||||
SchemaProjectionMap map(ProjectionIdEnum from, ProjectionIdEnum to) const {
|
||||
int id_from = schema_id(from);
|
||||
int id_to = schema_id(to);
|
||||
SchemaProjectionMap result;
|
||||
result.num_cols = num_cols(from);
|
||||
result.source_to_base = mappings_[id_from].data();
|
||||
result.base_to_target = inverse_mappings_[id_to].data();
|
||||
return result;
|
||||
}
|
||||
|
||||
protected:
|
||||
struct FieldInfos {
|
||||
std::vector<int> field_paths;
|
||||
std::vector<std::string> field_names;
|
||||
std::vector<std::shared_ptr<DataType>> data_types;
|
||||
};
|
||||
|
||||
Status RegisterSchema(ProjectionIdEnum handle, const Schema& schema) {
|
||||
FieldInfos out_fields;
|
||||
const FieldVector& in_fields = schema.fields();
|
||||
out_fields.field_paths.resize(in_fields.size());
|
||||
out_fields.field_names.resize(in_fields.size());
|
||||
out_fields.data_types.resize(in_fields.size());
|
||||
for (size_t i = 0; i < in_fields.size(); ++i) {
|
||||
const std::string& name = in_fields[i]->name();
|
||||
const std::shared_ptr<DataType>& type = in_fields[i]->type();
|
||||
out_fields.field_paths[i] = static_cast<int>(i);
|
||||
out_fields.field_names[i] = name;
|
||||
out_fields.data_types[i] = type;
|
||||
}
|
||||
schemas_.push_back(std::make_pair(handle, out_fields));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status RegisterProjectedSchema(ProjectionIdEnum handle,
|
||||
const std::vector<FieldRef>& selected_fields,
|
||||
const Schema& full_schema) {
|
||||
FieldInfos out_fields;
|
||||
const FieldVector& in_fields = full_schema.fields();
|
||||
out_fields.field_paths.resize(selected_fields.size());
|
||||
out_fields.field_names.resize(selected_fields.size());
|
||||
out_fields.data_types.resize(selected_fields.size());
|
||||
for (size_t i = 0; i < selected_fields.size(); ++i) {
|
||||
// All fields must be found in schema without ambiguity
|
||||
ARROW_ASSIGN_OR_RAISE(auto match, selected_fields[i].FindOne(full_schema));
|
||||
const std::string& name = in_fields[match[0]]->name();
|
||||
const std::shared_ptr<DataType>& type = in_fields[match[0]]->type();
|
||||
out_fields.field_paths[i] = match[0];
|
||||
out_fields.field_names[i] = name;
|
||||
out_fields.data_types[i] = type;
|
||||
}
|
||||
schemas_.push_back(std::make_pair(handle, out_fields));
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void RegisterEnd() {
|
||||
size_t size = schemas_.size();
|
||||
mappings_.resize(size);
|
||||
inverse_mappings_.resize(size);
|
||||
int id_base = 0;
|
||||
for (size_t i = 0; i < size; ++i) {
|
||||
GenerateMapForProjection(static_cast<int>(i), id_base);
|
||||
}
|
||||
}
|
||||
|
||||
int schema_id(ProjectionIdEnum schema_handle) const {
|
||||
for (size_t i = 0; i < schemas_.size(); ++i) {
|
||||
if (schemas_[i].first == schema_handle) {
|
||||
return static_cast<int>(i);
|
||||
}
|
||||
}
|
||||
// We should never get here
|
||||
assert(false);
|
||||
return -1;
|
||||
}
|
||||
|
||||
void GenerateMapForProjection(int id_proj, int id_base) {
|
||||
int num_cols_proj = static_cast<int>(schemas_[id_proj].second.data_types.size());
|
||||
int num_cols_base = static_cast<int>(schemas_[id_base].second.data_types.size());
|
||||
|
||||
std::vector<int>& mapping = mappings_[id_proj];
|
||||
std::vector<int>& inverse_mapping = inverse_mappings_[id_proj];
|
||||
mapping.resize(num_cols_proj);
|
||||
inverse_mapping.resize(num_cols_base);
|
||||
|
||||
if (id_proj == id_base) {
|
||||
for (int i = 0; i < num_cols_base; ++i) {
|
||||
mapping[i] = inverse_mapping[i] = i;
|
||||
}
|
||||
} else {
|
||||
const FieldInfos& fields_proj = schemas_[id_proj].second;
|
||||
const FieldInfos& fields_base = schemas_[id_base].second;
|
||||
for (int i = 0; i < num_cols_base; ++i) {
|
||||
inverse_mapping[i] = SchemaProjectionMap::kMissingField;
|
||||
}
|
||||
for (int i = 0; i < num_cols_proj; ++i) {
|
||||
int field_id = SchemaProjectionMap::kMissingField;
|
||||
for (int j = 0; j < num_cols_base; ++j) {
|
||||
if (fields_proj.field_paths[i] == fields_base.field_paths[j]) {
|
||||
field_id = j;
|
||||
// If there are multiple matches for the same input field,
|
||||
// it will be mapped to the first match.
|
||||
break;
|
||||
}
|
||||
}
|
||||
assert(field_id != SchemaProjectionMap::kMissingField);
|
||||
mapping[i] = field_id;
|
||||
inverse_mapping[field_id] = i;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// vector used as a mapping from ProjectionIdEnum to fields
|
||||
std::vector<std::pair<ProjectionIdEnum, FieldInfos>> schemas_;
|
||||
std::vector<std::vector<int>> mappings_;
|
||||
std::vector<std::vector<int>> inverse_mappings_;
|
||||
};
|
||||
|
||||
using HashJoinProjectionMaps = SchemaProjectionMaps<HashJoinProjection>;
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,102 @@
|
||||
// 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 <atomic>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/status.h"
|
||||
#include "arrow/util/config.h"
|
||||
#include "arrow/util/logging.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
|
||||
// Atomic value surrounded by padding bytes to avoid cache line invalidation
|
||||
// whenever it is modified by a concurrent thread on a different CPU core.
|
||||
//
|
||||
template <typename T>
|
||||
class AtomicWithPadding {
|
||||
private:
|
||||
static constexpr int kCacheLineSize = 64;
|
||||
uint8_t padding_before[kCacheLineSize];
|
||||
|
||||
public:
|
||||
std::atomic<T> value;
|
||||
|
||||
private:
|
||||
uint8_t padding_after[kCacheLineSize];
|
||||
};
|
||||
|
||||
// Used for asynchronous execution of operations that can be broken into
|
||||
// a fixed number of symmetric tasks that can be executed concurrently.
|
||||
//
|
||||
// Implements priorities between multiple such operations, called task groups.
|
||||
//
|
||||
// Allows to specify the maximum number of in-flight tasks at any moment.
|
||||
//
|
||||
// Also allows for executing next pending tasks immediately using a caller thread.
|
||||
//
|
||||
class ARROW_ACERO_EXPORT TaskScheduler {
|
||||
public:
|
||||
using TaskImpl = std::function<Status(size_t, int64_t)>;
|
||||
using TaskGroupContinuationImpl = std::function<Status(size_t)>;
|
||||
using ScheduleImpl = std::function<Status(TaskGroupContinuationImpl)>;
|
||||
using AbortContinuationImpl = std::function<void()>;
|
||||
|
||||
virtual ~TaskScheduler() = default;
|
||||
|
||||
// Order in which task groups are registered represents priorities of their tasks
|
||||
// (the first group has the highest priority).
|
||||
//
|
||||
// Returns task group identifier that is used to request operations on the task group.
|
||||
virtual int RegisterTaskGroup(TaskImpl task_impl,
|
||||
TaskGroupContinuationImpl cont_impl) = 0;
|
||||
|
||||
virtual void RegisterEnd() = 0;
|
||||
|
||||
// total_num_tasks may be zero, in which case task group continuation will be executed
|
||||
// immediately
|
||||
virtual Status StartTaskGroup(size_t thread_id, int group_id,
|
||||
int64_t total_num_tasks) = 0;
|
||||
|
||||
// Execute given number of tasks immediately using caller thread
|
||||
virtual Status ExecuteMore(size_t thread_id, int num_tasks_to_execute,
|
||||
bool execute_all) = 0;
|
||||
|
||||
// Begin scheduling tasks using provided callback and
|
||||
// the limit on the number of in-flight tasks at any moment.
|
||||
//
|
||||
// Scheduling will continue as long as there are waiting tasks.
|
||||
//
|
||||
// It will automatically resume whenever new task group gets started.
|
||||
virtual Status StartScheduling(size_t thread_id, ScheduleImpl schedule_impl,
|
||||
int num_concurrent_tasks, bool use_sync_execution) = 0;
|
||||
|
||||
// Abort scheduling and execution.
|
||||
// Used in case of being notified about unrecoverable error for the entire query.
|
||||
virtual void Abort(AbortContinuationImpl impl) = 0;
|
||||
|
||||
static std::unique_ptr<TaskScheduler> Make();
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,86 @@
|
||||
// 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 <string>
|
||||
|
||||
#include "arrow/acero/options.h"
|
||||
#include "arrow/acero/test_util_internal.h"
|
||||
#include "arrow/testing/random.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
|
||||
// \brief Make a delaying source that is optionally noisy (prints when it emits)
|
||||
AsyncGenerator<std::optional<ExecBatch>> MakeDelayedGen(
|
||||
Iterator<std::optional<ExecBatch>> src, std::string label, double delay_sec,
|
||||
bool noisy = false);
|
||||
|
||||
// \brief Make a delaying source that is optionally noisy (prints when it emits)
|
||||
AsyncGenerator<std::optional<ExecBatch>> MakeDelayedGen(
|
||||
AsyncGenerator<std::optional<ExecBatch>> src, std::string label, double delay_sec,
|
||||
bool noisy = false);
|
||||
|
||||
// \brief Make a delaying source that is optionally noisy (prints when it emits)
|
||||
AsyncGenerator<std::optional<ExecBatch>> MakeDelayedGen(BatchesWithSchema src,
|
||||
std::string label,
|
||||
double delay_sec,
|
||||
bool noisy = false);
|
||||
|
||||
/// A node that slightly resequences the input at random
|
||||
struct JitterNodeOptions : public ExecNodeOptions {
|
||||
random::SeedType seed;
|
||||
/// The max amount to add to a node's "cost".
|
||||
int max_jitter_modifier;
|
||||
|
||||
explicit JitterNodeOptions(random::SeedType seed, int max_jitter_modifier = 5)
|
||||
: seed(seed), max_jitter_modifier(max_jitter_modifier) {}
|
||||
static constexpr std::string_view kName = "jitter";
|
||||
};
|
||||
|
||||
class GateImpl;
|
||||
|
||||
class Gate {
|
||||
public:
|
||||
static std::shared_ptr<Gate> Make();
|
||||
|
||||
Gate();
|
||||
virtual ~Gate();
|
||||
|
||||
void ReleaseAllBatches();
|
||||
void ReleaseOneBatch();
|
||||
Future<> WaitForNextReleasedBatch();
|
||||
|
||||
private:
|
||||
ARROW_DISALLOW_COPY_AND_ASSIGN(Gate);
|
||||
|
||||
GateImpl* impl_;
|
||||
};
|
||||
|
||||
// A node that holds all input batches until a given gate is released
|
||||
struct GatedNodeOptions : public ExecNodeOptions {
|
||||
explicit GatedNodeOptions(Gate* gate) : gate(gate) {}
|
||||
Gate* gate;
|
||||
|
||||
static constexpr std::string_view kName = "gated";
|
||||
};
|
||||
|
||||
void RegisterTestNodes();
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,31 @@
|
||||
// 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/record_batch.h"
|
||||
#include "arrow/type_traits.h"
|
||||
|
||||
namespace arrow::acero {
|
||||
|
||||
// normalize the value to unsigned 64-bits while preserving ordering of values
|
||||
template <typename T, enable_if_t<std::is_integral<T>::value, bool> = true>
|
||||
uint64_t NormalizeTime(T t);
|
||||
|
||||
uint64_t GetTime(const RecordBatch* batch, Type::type time_type, int col, uint64_t row);
|
||||
|
||||
} // namespace arrow::acero
|
||||
@@ -0,0 +1,65 @@
|
||||
// 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 <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/type_fwd.h"
|
||||
#include "arrow/acero/visibility.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/status.h"
|
||||
|
||||
namespace arrow {
|
||||
namespace acero {
|
||||
namespace internal {
|
||||
|
||||
class ARROW_ACERO_EXPORT TpchGen {
|
||||
public:
|
||||
virtual ~TpchGen() = default;
|
||||
|
||||
/*
|
||||
* \brief Create a factory for nodes that generate TPC-H data
|
||||
*
|
||||
* Note: Individual tables will reference each other. It is important that you only
|
||||
* create a single TpchGen instance for each plan and then you can create nodes for each
|
||||
* table from that single TpchGen instance. Note: Every batch will be scheduled as a new
|
||||
* task using the ExecPlan's scheduler.
|
||||
*/
|
||||
static Result<std::unique_ptr<TpchGen>> Make(
|
||||
ExecPlan* plan, double scale_factor = 1.0, int64_t batch_size = 4096,
|
||||
std::optional<int64_t> seed = std::nullopt);
|
||||
|
||||
// The below methods will create and add an ExecNode to the plan that generates
|
||||
// data for the desired table. If columns is empty, all columns will be generated.
|
||||
// The methods return the added ExecNode, which should be used for inputs.
|
||||
virtual Result<ExecNode*> Supplier(std::vector<std::string> columns = {}) = 0;
|
||||
virtual Result<ExecNode*> Part(std::vector<std::string> columns = {}) = 0;
|
||||
virtual Result<ExecNode*> PartSupp(std::vector<std::string> columns = {}) = 0;
|
||||
virtual Result<ExecNode*> Customer(std::vector<std::string> columns = {}) = 0;
|
||||
virtual Result<ExecNode*> Orders(std::vector<std::string> columns = {}) = 0;
|
||||
virtual Result<ExecNode*> Lineitem(std::vector<std::string> columns = {}) = 0;
|
||||
virtual Result<ExecNode*> Nation(std::vector<std::string> columns = {}) = 0;
|
||||
virtual Result<ExecNode*> Region(std::vector<std::string> columns = {}) = 0;
|
||||
};
|
||||
|
||||
} // namespace internal
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,36 @@
|
||||
// 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/type_fwd.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
namespace acero {
|
||||
|
||||
class ExecNode;
|
||||
class ExecPlan;
|
||||
class ExecNodeOptions;
|
||||
class ExecFactoryRegistry;
|
||||
class QueryContext;
|
||||
struct QueryOptions;
|
||||
struct Declaration;
|
||||
class SinkNodeConsumer;
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,184 @@
|
||||
// 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 <atomic>
|
||||
#include <cstdint>
|
||||
#include <optional>
|
||||
#include <thread>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
|
||||
#include "arrow/acero/options.h"
|
||||
#include "arrow/acero/type_fwd.h"
|
||||
#include "arrow/buffer.h"
|
||||
#include "arrow/compute/expression.h"
|
||||
#include "arrow/compute/util.h"
|
||||
#include "arrow/memory_pool.h"
|
||||
#include "arrow/result.h"
|
||||
#include "arrow/status.h"
|
||||
#include "arrow/util/bit_util.h"
|
||||
#include "arrow/util/cpu_info.h"
|
||||
#include "arrow/util/logging.h"
|
||||
#include "arrow/util/mutex.h"
|
||||
#include "arrow/util/thread_pool.h"
|
||||
#include "arrow/util/type_fwd.h"
|
||||
|
||||
namespace arrow {
|
||||
|
||||
namespace acero {
|
||||
|
||||
ARROW_ACERO_EXPORT
|
||||
Status ValidateExecNodeInputs(ExecPlan* plan, const std::vector<ExecNode*>& inputs,
|
||||
int expected_num_inputs, const char* kind_name);
|
||||
|
||||
ARROW_ACERO_EXPORT
|
||||
Result<std::shared_ptr<Table>> TableFromExecBatches(
|
||||
const std::shared_ptr<Schema>& schema, const std::vector<ExecBatch>& exec_batches);
|
||||
|
||||
class ARROW_ACERO_EXPORT AtomicCounter {
|
||||
public:
|
||||
AtomicCounter() = default;
|
||||
|
||||
int count() const { return count_.load(); }
|
||||
|
||||
std::optional<int> total() const {
|
||||
int total = total_.load();
|
||||
if (total == -1) return {};
|
||||
return total;
|
||||
}
|
||||
|
||||
// return true if the counter is complete
|
||||
bool Increment() {
|
||||
ARROW_DCHECK_NE(count_.load(), total_.load());
|
||||
int count = count_.fetch_add(1) + 1;
|
||||
if (count != total_.load()) return false;
|
||||
return DoneOnce();
|
||||
}
|
||||
|
||||
// return true if the counter is complete
|
||||
bool SetTotal(int total) {
|
||||
total_.store(total);
|
||||
if (count_.load() != total) return false;
|
||||
return DoneOnce();
|
||||
}
|
||||
|
||||
// return true if the counter has not already been completed
|
||||
bool Cancel() { return DoneOnce(); }
|
||||
|
||||
// return true if the counter has finished or been cancelled
|
||||
bool Completed() { return complete_.load(); }
|
||||
|
||||
private:
|
||||
// ensure there is only one true return from Increment(), SetTotal(), or Cancel()
|
||||
bool DoneOnce() {
|
||||
bool expected = false;
|
||||
return complete_.compare_exchange_strong(expected, true);
|
||||
}
|
||||
|
||||
std::atomic<int> count_{0}, total_{-1};
|
||||
std::atomic<bool> complete_{false};
|
||||
};
|
||||
|
||||
class ARROW_ACERO_EXPORT ThreadIndexer {
|
||||
public:
|
||||
size_t operator()();
|
||||
|
||||
static size_t Capacity();
|
||||
|
||||
private:
|
||||
static size_t Check(size_t thread_index);
|
||||
|
||||
arrow::util::Mutex mutex_;
|
||||
std::unordered_map<std::thread::id, size_t> id_to_index_;
|
||||
};
|
||||
|
||||
/// \brief A consumer that collects results into an in-memory table
|
||||
struct ARROW_ACERO_EXPORT TableSinkNodeConsumer : public SinkNodeConsumer {
|
||||
public:
|
||||
TableSinkNodeConsumer(std::shared_ptr<Table>* out, MemoryPool* pool)
|
||||
: out_(out), pool_(pool) {}
|
||||
Status Init(const std::shared_ptr<Schema>& schema,
|
||||
BackpressureControl* backpressure_control, ExecPlan* plan) override;
|
||||
Status Consume(ExecBatch batch) override;
|
||||
Future<> Finish() override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<Table>* out_;
|
||||
MemoryPool* pool_;
|
||||
std::shared_ptr<Schema> schema_;
|
||||
std::vector<std::shared_ptr<RecordBatch>> batches_;
|
||||
arrow::util::Mutex consume_mutex_;
|
||||
};
|
||||
|
||||
class ARROW_ACERO_EXPORT NullSinkNodeConsumer : public SinkNodeConsumer {
|
||||
public:
|
||||
Status Init(const std::shared_ptr<Schema>&, BackpressureControl*,
|
||||
ExecPlan* plan) override {
|
||||
return Status::OK();
|
||||
}
|
||||
Status Consume(ExecBatch exec_batch) override { return Status::OK(); }
|
||||
Future<> Finish() override { return Status::OK(); }
|
||||
|
||||
public:
|
||||
static std::shared_ptr<NullSinkNodeConsumer> Make() {
|
||||
return std::make_shared<NullSinkNodeConsumer>();
|
||||
}
|
||||
};
|
||||
|
||||
/// CRTP helper for tracing helper functions
|
||||
|
||||
class ARROW_ACERO_EXPORT TracedNode {
|
||||
public:
|
||||
// All nodes should call TraceStartProducing or NoteStartProducing exactly once
|
||||
// Most nodes will be fine with a call to NoteStartProducing since the StartProducing
|
||||
// call is usually fairly cheap and simply schedules tasks to fetch the actual data.
|
||||
|
||||
explicit TracedNode(ExecNode* node) : node_(node) {}
|
||||
|
||||
// Create a span to record the StartProducing work
|
||||
[[nodiscard]] ::arrow::internal::tracing::Scope TraceStartProducing(
|
||||
std::string extra_details) const;
|
||||
|
||||
// Record a call to StartProducing without creating with a span
|
||||
void NoteStartProducing(std::string extra_details) const;
|
||||
|
||||
// All nodes should call TraceInputReceived for each batch they receive. This call
|
||||
// should track the time spent processing the batch. NoteInputReceived is available
|
||||
// but usually won't be used unless a node is simply adding batches to a trivial queue.
|
||||
|
||||
// Create a span to record the InputReceived work
|
||||
[[nodiscard]] ::arrow::internal::tracing::Scope TraceInputReceived(
|
||||
const ExecBatch& batch) const;
|
||||
|
||||
// Record a call to InputReceived without creating with a span
|
||||
void NoteInputReceived(const ExecBatch& batch) const;
|
||||
|
||||
// Create a span to record any "finish" work. This should NOT be called as part of
|
||||
// InputFinished and many nodes may not need to call this at all. This should be used
|
||||
// when a node has some extra work that has to be done once it has received all of its
|
||||
// data. For example, an aggregation node calculating aggregations. This will
|
||||
// typically be called as a result of InputFinished OR InputReceived.
|
||||
[[nodiscard]] ::arrow::internal::tracing::Scope TraceFinish() const;
|
||||
|
||||
private:
|
||||
ExecNode* node_;
|
||||
};
|
||||
|
||||
} // namespace acero
|
||||
} // namespace arrow
|
||||
@@ -0,0 +1,50 @@
|
||||
// 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.
|
||||
|
||||
// This API is EXPERIMENTAL.
|
||||
|
||||
#pragma once
|
||||
|
||||
#if defined(_WIN32) || defined(__CYGWIN__)
|
||||
# if defined(_MSC_VER)
|
||||
# pragma warning(push)
|
||||
# pragma warning(disable : 4251)
|
||||
# else
|
||||
# pragma GCC diagnostic ignored "-Wattributes"
|
||||
# endif
|
||||
|
||||
# ifdef ARROW_ACERO_STATIC
|
||||
# define ARROW_ACERO_EXPORT
|
||||
# elif defined(ARROW_ACERO_EXPORTING)
|
||||
# define ARROW_ACERO_EXPORT __declspec(dllexport)
|
||||
# else
|
||||
# define ARROW_ACERO_EXPORT __declspec(dllimport)
|
||||
# endif
|
||||
|
||||
# define ARROW_ACERO_NO_EXPORT
|
||||
#else // Not Windows
|
||||
# ifndef ARROW_ACERO_EXPORT
|
||||
# define ARROW_ACERO_EXPORT __attribute__((visibility("default")))
|
||||
# endif
|
||||
# ifndef ARROW_ACERO_NO_EXPORT
|
||||
# define ARROW_ACERO_NO_EXPORT __attribute__((visibility("hidden")))
|
||||
# endif
|
||||
#endif // Not-Windows
|
||||
|
||||
#if defined(_MSC_VER)
|
||||
# pragma warning(pop)
|
||||
#endif
|
||||
Reference in New Issue
Block a user