diff --git a/cpp/src/arrow/compute/exec.cc b/cpp/src/arrow/compute/exec.cc index b88248071c2..6df845f0f44 100644 --- a/cpp/src/arrow/compute/exec.cc +++ b/cpp/src/arrow/compute/exec.cc @@ -616,8 +616,7 @@ class ScalarExecutor : public KernelExecutorImpl { } } - kernel_->exec(kernel_ctx_, batch, &out); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + RETURN_NOT_OK(kernel_->exec(kernel_ctx_, batch, &out)); if (!preallocate_contiguous_) { // If we are producing chunked output rather than one big array, then // emit each chunk as soon as it's available @@ -794,8 +793,7 @@ class VectorExecutor : public KernelExecutorImpl { output_descr_.shape == ValueDescr::ARRAY) { RETURN_NOT_OK(PropagateNulls(kernel_ctx_, batch, out.mutable_array())); } - kernel_->exec(kernel_ctx_, batch, &out); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + RETURN_NOT_OK(kernel_->exec(kernel_ctx_, batch, &out)); if (!kernel_->finalize) { // If there is no result finalizer (e.g. for hash-based functions, we can // emit the processed batch right away rather than waiting @@ -810,8 +808,7 @@ class VectorExecutor : public KernelExecutorImpl { if (kernel_->finalize) { // Intermediate results require post-processing after the execution is // completed (possibly involving some accumulated state) - kernel_->finalize(kernel_ctx_, &results_); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + RETURN_NOT_OK(kernel_->finalize(kernel_ctx_, &results_)); for (const auto& result : results_) { RETURN_NOT_OK(listener->OnResult(result)); } @@ -864,8 +861,7 @@ class ScalarAggExecutor : public KernelExecutorImpl { } Datum out; - kernel_->finalize(kernel_ctx_, &out); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + RETURN_NOT_OK(kernel_->finalize(kernel_ctx_, &out)); RETURN_NOT_OK(listener->OnResult(std::move(out))); return Status::OK(); } @@ -879,24 +875,19 @@ class ScalarAggExecutor : public KernelExecutorImpl { private: Status Consume(const ExecBatch& batch) { // FIXME(ARROW-11840) don't merge *any* aggegates for every batch - auto batch_state = kernel_->init(kernel_ctx_, {kernel_, *input_descrs_, options_}); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); + ARROW_ASSIGN_OR_RAISE( + auto batch_state, + kernel_->init(kernel_ctx_, {kernel_, *input_descrs_, options_})); if (batch_state == nullptr) { - kernel_ctx_->SetStatus( - Status::Invalid("ScalarAggregation requires non-null kernel state")); - return kernel_ctx_->status(); + return Status::Invalid("ScalarAggregation requires non-null kernel state"); } KernelContext batch_ctx(exec_context()); batch_ctx.SetState(batch_state.get()); - kernel_->consume(&batch_ctx, batch); - ARROW_CTX_RETURN_IF_ERROR(&batch_ctx); - - kernel_->merge(kernel_ctx_, std::move(*batch_state), state()); - ARROW_CTX_RETURN_IF_ERROR(kernel_ctx_); - + RETURN_NOT_OK(kernel_->consume(&batch_ctx, batch)); + RETURN_NOT_OK(kernel_->merge(kernel_ctx_, std::move(*batch_state), state())); return Status::OK(); } diff --git a/cpp/src/arrow/compute/exec_test.cc b/cpp/src/arrow/compute/exec_test.cc index e9bd57596b5..c56e6471c97 100644 --- a/cpp/src/arrow/compute/exec_test.cc +++ b/cpp/src/arrow/compute/exec_test.cc @@ -537,7 +537,7 @@ TEST_F(TestExecBatchIterator, ZeroLengthInputs) { // ---------------------------------------------------------------------- // Scalar function execution -void ExecCopy(KernelContext*, const ExecBatch& batch, Datum* out) { +Status ExecCopy(KernelContext*, const ExecBatch& batch, Datum* out) { DCHECK_EQ(1, batch.num_values()); const auto& type = checked_cast(*batch[0].type()); int value_size = type.bit_width() / 8; @@ -547,9 +547,10 @@ void ExecCopy(KernelContext*, const ExecBatch& batch, Datum* out) { uint8_t* dst = out_arr->buffers[1]->mutable_data() + out_arr->offset * value_size; const uint8_t* src = arg0.buffers[1]->data() + arg0.offset * value_size; std::memcpy(dst, src, batch.length * value_size); + return Status::OK(); } -void ExecComputedBitmap(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ExecComputedBitmap(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // Propagate nulls not used. Check that the out bitmap isn't the same already // as the input bitmap const ArrayData& arg0 = *batch[0].array(); @@ -564,10 +565,10 @@ void ExecComputedBitmap(KernelContext* ctx, const ExecBatch& batch, Datum* out) internal::CopyBitmap(arg0.buffers[0]->data(), arg0.offset, batch.length, out_arr->buffers[0]->mutable_data(), out_arr->offset); - ExecCopy(ctx, batch, out); + return ExecCopy(ctx, batch, out); } -void ExecNoPreallocatedData(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ExecNoPreallocatedData(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // Validity preallocated, but not the data ArrayData* out_arr = out->mutable_array(); DCHECK_EQ(0, out_arr->offset); @@ -575,10 +576,11 @@ void ExecNoPreallocatedData(KernelContext* ctx, const ExecBatch& batch, Datum* o int value_size = type.bit_width() / 8; Status s = (ctx->Allocate(out_arr->length * value_size).Value(&out_arr->buffers[1])); DCHECK_OK(s); - ExecCopy(ctx, batch, out); + return ExecCopy(ctx, batch, out); } -void ExecNoPreallocatedAnything(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ExecNoPreallocatedAnything(KernelContext* ctx, const ExecBatch& batch, + Datum* out) { // Neither validity nor data preallocated ArrayData* out_arr = out->mutable_array(); DCHECK_EQ(0, out_arr->offset); @@ -589,7 +591,7 @@ void ExecNoPreallocatedAnything(KernelContext* ctx, const ExecBatch& batch, Datu out_arr->buffers[0]->mutable_data(), /*offset=*/0); // Reuse the kernel that allocates the data - ExecNoPreallocatedData(ctx, batch, out); + return ExecNoPreallocatedData(ctx, batch, out); } struct ExampleOptions : public FunctionOptions { @@ -602,12 +604,13 @@ struct ExampleState : public KernelState { explicit ExampleState(std::shared_ptr value) : value(std::move(value)) {} }; -std::unique_ptr InitStateful(KernelContext*, const KernelInitArgs& args) { +Result> InitStateful(KernelContext*, + const KernelInitArgs& args) { auto func_options = static_cast(args.options); return std::unique_ptr(new ExampleState{func_options->value}); } -void ExecStateful(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ExecStateful(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // We take the value from the state and multiply the data in batch[0] with it ExampleState* state = static_cast(ctx->state()); int32_t multiplier = checked_cast(*state->value).value; @@ -619,12 +622,14 @@ void ExecStateful(KernelContext* ctx, const ExecBatch& batch, Datum* out) { for (int64_t i = 0; i < arg0.length; ++i) { dst[i] = arg0_data[i] * multiplier; } + return Status::OK(); } -void ExecAddInt32(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ExecAddInt32(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const Int32Scalar& arg0 = batch[0].scalar_as(); const Int32Scalar& arg1 = batch[1].scalar_as(); out->value = std::make_shared(arg0.value + arg1.value); + return Status::OK(); } class TestCallScalarFunction : public TestComputeInternals { diff --git a/cpp/src/arrow/compute/function.cc b/cpp/src/arrow/compute/function.cc index c8fc8b8dec0..f74bb245d77 100644 --- a/cpp/src/arrow/compute/function.cc +++ b/cpp/src/arrow/compute/function.cc @@ -179,8 +179,7 @@ Result Function::Execute(const std::vector& args, KernelContext kernel_ctx{ctx}; if (kernel->init) { - state = kernel->init(&kernel_ctx, {kernel, inputs, options}); - RETURN_NOT_OK(kernel_ctx.status()); + ARROW_ASSIGN_OR_RAISE(state, kernel->init(&kernel_ctx, {kernel, inputs, options})); kernel_ctx.SetState(state.get()); } diff --git a/cpp/src/arrow/compute/function_benchmark.cc b/cpp/src/arrow/compute/function_benchmark.cc index 5dc305bdd89..daf03754984 100644 --- a/cpp/src/arrow/compute/function_benchmark.cc +++ b/cpp/src/arrow/compute/function_benchmark.cc @@ -78,16 +78,17 @@ void BM_CastDispatchBaseline(benchmark::State& state) { ExecContext exec_context; KernelContext kernel_context(&exec_context); - auto cast_state = - cast_kernel->init(&kernel_context, {cast_kernel, {double_type}, &cast_options}); - ABORT_NOT_OK(kernel_context.status()); + auto cast_state = cast_kernel + ->init(&kernel_context, + KernelInitArgs{cast_kernel, {double_type}, &cast_options}) + .ValueOrDie(); kernel_context.SetState(cast_state.get()); for (auto _ : state) { Datum timestamp_scalar = MakeNullScalar(double_type); for (Datum int_scalar : int_scalars) { - exec(&kernel_context, {{std::move(int_scalar)}, 1}, ×tamp_scalar); - ABORT_NOT_OK(kernel_context.status()); + ABORT_NOT_OK( + exec(&kernel_context, {{std::move(int_scalar)}, 1}, ×tamp_scalar)); } benchmark::DoNotOptimize(timestamp_scalar); } @@ -164,8 +165,7 @@ void BM_ExecuteScalarKernelOnScalar(benchmark::State& state) { int64_t total = 0; for (const auto& scalar : scalars) { Datum result{MakeNullScalar(int64())}; - exec(&kernel_context, ExecBatch{{scalar}, /*length=*/1}, &result); - ABORT_NOT_OK(kernel_context.status()); + ABORT_NOT_OK(exec(&kernel_context, ExecBatch{{scalar}, /*length=*/1}, &result)); total += result.scalar()->is_valid; } benchmark::DoNotOptimize(total); diff --git a/cpp/src/arrow/compute/function_test.cc b/cpp/src/arrow/compute/function_test.cc index b6f1815b89e..581555e931f 100644 --- a/cpp/src/arrow/compute/function_test.cc +++ b/cpp/src/arrow/compute/function_test.cc @@ -87,8 +87,7 @@ TEST(VectorFunction, Basics) { } auto ExecNYI = [](KernelContext* ctx, const ExecBatch& args, Datum* out) { - ctx->SetStatus(Status::NotImplemented("NYI")); - return; + return Status::NotImplemented("NYI"); }; template @@ -181,13 +180,15 @@ TEST(ScalarAggregateFunction, Basics) { ASSERT_EQ(Function::SCALAR_AGGREGATE, func.kind()); } -std::unique_ptr NoopInit(KernelContext*, const KernelInitArgs&) { +Result> NoopInit(KernelContext*, const KernelInitArgs&) { return nullptr; } -void NoopConsume(KernelContext*, const ExecBatch&) {} -void NoopMerge(KernelContext*, const KernelState&, KernelState*) {} -void NoopFinalize(KernelContext*, Datum*) {} +Status NoopConsume(KernelContext*, const ExecBatch&) { return Status::OK(); } +Status NoopMerge(KernelContext*, const KernelState&, KernelState*) { + return Status::OK(); +} +Status NoopFinalize(KernelContext*, Datum*) { return Status::OK(); } TEST(ScalarAggregateFunction, DispatchExact) { ScalarAggregateFunction func("agg_test", Arity::Unary(), /*doc=*/nullptr); diff --git a/cpp/src/arrow/compute/kernel.cc b/cpp/src/arrow/compute/kernel.cc index 88b42716fa2..8fa740ed247 100644 --- a/cpp/src/arrow/compute/kernel.cc +++ b/cpp/src/arrow/compute/kernel.cc @@ -59,16 +59,6 @@ Result> KernelContext::AllocateBitmap(int64_t n return result; } -void KernelContext::SetStatus(const Status& status) { - if (ARROW_PREDICT_TRUE(status.ok())) { - return; - } - status_ = status; -} - -/// \brief Clear any error status -void KernelContext::ResetStatus() { status_ = Status::OK(); } - // ---------------------------------------------------------------------- // Some basic TypeMatcher implementations diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h index b99b41170d2..0fecea080d8 100644 --- a/cpp/src/arrow/compute/kernel.h +++ b/cpp/src/arrow/compute/kernel.h @@ -63,22 +63,6 @@ class ARROW_EXPORT KernelContext { /// byte is preemptively zeroed to help avoid ASAN or valgrind issues. Result> AllocateBitmap(int64_t num_bits); - /// \brief Indicate that an error has occurred, to be checked by a exec caller - /// \param[in] status a Status instance. - /// - /// \note Will not overwrite a prior set Status, so we will have the first - /// error that occurred until ExecContext::ResetStatus is called. - void SetStatus(const Status& status); - - /// \brief Clear any error status. - void ResetStatus(); - - /// \brief Return true if an error has occurred. - bool HasError() const { return !status_.ok(); } - - /// \brief Return the current status of the context. - const Status& status() const { return status_; } - /// \brief Assign the active KernelState to be utilized for each stage of /// kernel execution. Ownership and memory lifetime of the KernelState must /// be minded separately. @@ -96,21 +80,9 @@ class ARROW_EXPORT KernelContext { private: ExecContext* exec_ctx_; - Status status_; KernelState* state_; }; -// A macro to invoke for error control flow after invoking functions (such as -// kernel init or exec functions) that propagate errors via KernelContext. -#define ARROW_CTX_RETURN_IF_ERROR(CTX) \ - do { \ - if (ARROW_PREDICT_FALSE((CTX)->HasError())) { \ - Status s = (CTX)->status(); \ - (CTX)->ResetStatus(); \ - return s; \ - } \ - } while (0) - /// \brief The standard kernel execution API that must be implemented for /// SCALAR and VECTOR kernel types. This includes both stateless and stateful /// kernels. Kernels depending on some execution state access that state via @@ -119,7 +91,7 @@ class ARROW_EXPORT KernelContext { /// into pre-allocated memory if they are able, though for some kernels /// (e.g. in cases when a builder like StringBuilder) must be employed this may /// not be possible. -using ArrayKernelExec = std::function; +using ArrayKernelExec = std::function; /// \brief An type-checking interface to permit customizable validation rules /// for use with InputType and KernelSignature. This is for scenarios where the @@ -523,9 +495,8 @@ struct KernelInitArgs { }; /// \brief Common initializer function for all kernel types. -/// If an error occurs it will be stored in the KernelContext; nullptr will be returned. -using KernelInit = - std::function(KernelContext*, const KernelInitArgs&)>; +using KernelInit = std::function>( + KernelContext*, const KernelInitArgs&)>; /// \brief Base type for kernels. Contains the function signature and /// optionally the state initialization function, along with some common @@ -608,7 +579,7 @@ struct ScalarKernel : public ArrayKernel { // VectorKernel (for VectorFunction) /// \brief See VectorKernel::finalize member for usage -using VectorFinalize = std::function*)>; +using VectorFinalize = std::function*)>; /// \brief Kernel data structure for implementations of VectorFunction. In /// addition to the members found in ArrayKernel, contains an optional @@ -663,13 +634,13 @@ struct VectorKernel : public ArrayKernel { // ---------------------------------------------------------------------- // ScalarAggregateKernel (for ScalarAggregateFunction) -using ScalarAggregateConsume = std::function; +using ScalarAggregateConsume = std::function; using ScalarAggregateMerge = - std::function; + std::function; // Finalize returns Datum to permit multiple return values -using ScalarAggregateFinalize = std::function; +using ScalarAggregateFinalize = std::function; /// \brief Kernel data structure for implementations of /// ScalarAggregateFunction. The four necessary components of an aggregation @@ -707,13 +678,13 @@ struct ScalarAggregateKernel : public Kernel { // ---------------------------------------------------------------------- // HashAggregateKernel (for HashAggregateFunction) -using HashAggregateConsume = std::function; +using HashAggregateConsume = std::function; using HashAggregateMerge = - std::function; + std::function; // Finalize returns Datum to permit multiple return values -using HashAggregateFinalize = std::function; +using HashAggregateFinalize = std::function; /// \brief Kernel data structure for implementations of /// HashAggregateFunction. The four necessary components of an aggregation diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic.cc b/cpp/src/arrow/compute/kernels/aggregate_basic.cc index 61dc8cb403c..e4eec50c66d 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic.cc @@ -27,16 +27,16 @@ namespace compute { namespace { -void AggregateConsume(KernelContext* ctx, const ExecBatch& batch) { - checked_cast(ctx->state())->Consume(ctx, batch); +Status AggregateConsume(KernelContext* ctx, const ExecBatch& batch) { + return checked_cast(ctx->state())->Consume(ctx, batch); } -void AggregateMerge(KernelContext* ctx, KernelState&& src, KernelState* dst) { - checked_cast(dst)->MergeFrom(ctx, std::move(src)); +Status AggregateMerge(KernelContext* ctx, KernelState&& src, KernelState* dst) { + return checked_cast(dst)->MergeFrom(ctx, std::move(src)); } -void AggregateFinalize(KernelContext* ctx, Datum* out) { - checked_cast(ctx->state())->Finalize(ctx, out); +Status AggregateFinalize(KernelContext* ctx, Datum* out) { + return checked_cast(ctx->state())->Finalize(ctx, out); } } // namespace @@ -58,20 +58,22 @@ namespace aggregate { struct CountImpl : public ScalarAggregator { explicit CountImpl(CountOptions options) : options(std::move(options)) {} - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { const ArrayData& input = *batch[0].array(); const int64_t nulls = input.GetNullCount(); this->nulls += nulls; this->non_nulls += input.length - nulls; + return Status::OK(); } - void MergeFrom(KernelContext*, KernelState&& src) override { + Status MergeFrom(KernelContext*, KernelState&& src) override { const auto& other_state = checked_cast(src); this->non_nulls += other_state.non_nulls; this->nulls += other_state.nulls; + return Status::OK(); } - void Finalize(KernelContext* ctx, Datum* out) override { + Status Finalize(KernelContext* ctx, Datum* out) override { const auto& state = checked_cast(*ctx->state()); switch (state.options.count_mode) { case CountOptions::COUNT_NON_NULL: @@ -81,9 +83,9 @@ struct CountImpl : public ScalarAggregator { *out = Datum(state.nulls); break; default: - ctx->SetStatus(Status::Invalid("Unknown CountOptions encountered")); - break; + return Status::Invalid("Unknown CountOptions encountered"); } + return Status::OK(); } CountOptions options; @@ -91,7 +93,8 @@ struct CountImpl : public ScalarAggregator { int64_t nulls = 0; }; -std::unique_ptr CountInit(KernelContext*, const KernelInitArgs& args) { +Result> CountInit(KernelContext*, + const KernelInitArgs& args) { return ::arrow::internal::make_unique( static_cast(*args.options)); } @@ -105,12 +108,14 @@ struct SumImplDefault : public SumImpl {}; template struct MeanImplDefault : public MeanImpl {}; -std::unique_ptr SumInit(KernelContext* ctx, const KernelInitArgs& args) { +Result> SumInit(KernelContext* ctx, + const KernelInitArgs& args) { SumLikeInit visitor(ctx, *args.inputs[0].type); return visitor.Create(); } -std::unique_ptr MeanInit(KernelContext* ctx, const KernelInitArgs& args) { +Result> MeanInit(KernelContext* ctx, + const KernelInitArgs& args) { SumLikeInit visitor(ctx, *args.inputs[0].type); return visitor.Create(); } @@ -118,7 +123,8 @@ std::unique_ptr MeanInit(KernelContext* ctx, const KernelInitArgs& // ---------------------------------------------------------------------- // MinMax implementation -std::unique_ptr MinMaxInit(KernelContext* ctx, const KernelInitArgs& args) { +Result> MinMaxInit(KernelContext* ctx, + const KernelInitArgs& args) { MinMaxInitState visitor( ctx, *args.inputs[0].type, args.kernel->signature->out_type().type(), static_cast(*args.options)); @@ -129,10 +135,10 @@ std::unique_ptr MinMaxInit(KernelContext* ctx, const KernelInitArgs // Any implementation struct BooleanAnyImpl : public ScalarAggregator { - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { // short-circuit if seen a True already if (this->any == true) { - return; + return Status::OK(); } const auto& data = *batch[0].array(); @@ -147,20 +153,24 @@ struct BooleanAnyImpl : public ScalarAggregator { } position += block.length; } + return Status::OK(); } - void MergeFrom(KernelContext*, KernelState&& src) override { + Status MergeFrom(KernelContext*, KernelState&& src) override { const auto& other = checked_cast(src); this->any |= other.any; + return Status::OK(); } - void Finalize(KernelContext*, Datum* out) override { + Status Finalize(KernelContext*, Datum* out) override { out->value = std::make_shared(this->any); + return Status::OK(); } + bool any = false; }; -std::unique_ptr AnyInit(KernelContext*, const KernelInitArgs& args) { +Result> AnyInit(KernelContext*, const KernelInitArgs& args) { return ::arrow::internal::make_unique(); } @@ -168,10 +178,10 @@ std::unique_ptr AnyInit(KernelContext*, const KernelInitArgs& args) // All implementation struct BooleanAllImpl : public ScalarAggregator { - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { // short-circuit if seen a false already if (this->all == false) { - return; + return Status::OK(); } const auto& data = *batch[0].array(); @@ -186,20 +196,25 @@ struct BooleanAllImpl : public ScalarAggregator { } position += block.length; } + + return Status::OK(); } - void MergeFrom(KernelContext*, KernelState&& src) override { + Status MergeFrom(KernelContext*, KernelState&& src) override { const auto& other = checked_cast(src); this->all &= other.all; + return Status::OK(); } - void Finalize(KernelContext*, Datum* out) override { + Status Finalize(KernelContext*, Datum* out) override { out->value = std::make_shared(this->all); + return Status::OK(); } + bool all = true; }; -std::unique_ptr AllInit(KernelContext*, const KernelInitArgs& args) { +Result> AllInit(KernelContext*, const KernelInitArgs& args) { return ::arrow::internal::make_unique(); } diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic_avx2.cc b/cpp/src/arrow/compute/kernels/aggregate_basic_avx2.cc index feeb66a1489..a70363aab9b 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic_avx2.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic_avx2.cc @@ -30,13 +30,14 @@ struct SumImplAvx2 : public SumImpl {}; template struct MeanImplAvx2 : public MeanImpl {}; -std::unique_ptr SumInitAvx2(KernelContext* ctx, const KernelInitArgs& args) { +Result> SumInitAvx2(KernelContext* ctx, + const KernelInitArgs& args) { SumLikeInit visitor(ctx, *args.inputs[0].type); return visitor.Create(); } -std::unique_ptr MeanInitAvx2(KernelContext* ctx, - const KernelInitArgs& args) { +Result> MeanInitAvx2(KernelContext* ctx, + const KernelInitArgs& args) { SumLikeInit visitor(ctx, *args.inputs[0].type); return visitor.Create(); } @@ -44,8 +45,8 @@ std::unique_ptr MeanInitAvx2(KernelContext* ctx, // ---------------------------------------------------------------------- // MinMax implementation -std::unique_ptr MinMaxInitAvx2(KernelContext* ctx, - const KernelInitArgs& args) { +Result> MinMaxInitAvx2(KernelContext* ctx, + const KernelInitArgs& args) { MinMaxInitState visitor( ctx, *args.inputs[0].type, args.kernel->signature->out_type().type(), static_cast(*args.options)); diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic_avx512.cc b/cpp/src/arrow/compute/kernels/aggregate_basic_avx512.cc index 522564a8469..1ecbd7041e6 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic_avx512.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_basic_avx512.cc @@ -30,14 +30,14 @@ struct SumImplAvx512 : public SumImpl {}; template struct MeanImplAvx512 : public MeanImpl {}; -std::unique_ptr SumInitAvx512(KernelContext* ctx, - const KernelInitArgs& args) { +Result> SumInitAvx512(KernelContext* ctx, + const KernelInitArgs& args) { SumLikeInit visitor(ctx, *args.inputs[0].type); return visitor.Create(); } -std::unique_ptr MeanInitAvx512(KernelContext* ctx, - const KernelInitArgs& args) { +Result> MeanInitAvx512(KernelContext* ctx, + const KernelInitArgs& args) { SumLikeInit visitor(ctx, *args.inputs[0].type); return visitor.Create(); } @@ -45,8 +45,8 @@ std::unique_ptr MeanInitAvx512(KernelContext* ctx, // ---------------------------------------------------------------------- // MinMax implementation -std::unique_ptr MinMaxInitAvx512(KernelContext* ctx, - const KernelInitArgs& args) { +Result> MinMaxInitAvx512(KernelContext* ctx, + const KernelInitArgs& args) { MinMaxInitState visitor( ctx, *args.inputs[0].type, args.kernel->signature->out_type().type(), static_cast(*args.options)); diff --git a/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h b/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h index 5029c1855c0..f8db180b1e3 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h +++ b/cpp/src/arrow/compute/kernels/aggregate_basic_internal.h @@ -58,7 +58,7 @@ struct SumImpl : public ScalarAggregator { using SumType = typename FindAccumulatorType::Type; using OutputType = typename TypeTraits::ScalarType; - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { const auto& data = batch[0].array(); this->count = data->length - data->GetNullCount(); if (is_boolean_type::value) { @@ -67,20 +67,23 @@ struct SumImpl : public ScalarAggregator { this->sum = arrow::compute::detail::SumArray(*data); } + return Status::OK(); } - void MergeFrom(KernelContext*, KernelState&& src) override { + Status MergeFrom(KernelContext*, KernelState&& src) override { const auto& other = checked_cast(src); this->count += other.count; this->sum += other.sum; + return Status::OK(); } - void Finalize(KernelContext*, Datum* out) override { + Status Finalize(KernelContext*, Datum* out) override { if (this->count == 0) { out->value = std::make_shared(); } else { out->value = MakeScalar(this->sum); } + return Status::OK(); } size_t count = 0; @@ -89,13 +92,14 @@ struct SumImpl : public ScalarAggregator { template struct MeanImpl : public SumImpl { - void Finalize(KernelContext*, Datum* out) override { + Status Finalize(KernelContext*, Datum* out) override { if (this->count == 0) { out->value = std::make_shared(); } else { const double mean = static_cast(this->sum) / this->count; out->value = std::make_shared(mean); } + return Status::OK(); } }; @@ -124,8 +128,8 @@ struct SumLikeInit { return Status::OK(); } - std::unique_ptr Create() { - ctx->SetStatus(VisitTypeInline(type, this)); + Result> Create() { + RETURN_NOT_OK(VisitTypeInline(type, this)); return std::move(state); } }; @@ -217,7 +221,7 @@ struct MinMaxImpl : public ScalarAggregator { MinMaxImpl(const std::shared_ptr& out_type, const MinMaxOptions& options) : out_type(out_type), options(options) {} - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { StateType local; ArrayType arr(batch[0].array()); @@ -228,7 +232,7 @@ struct MinMaxImpl : public ScalarAggregator { if (local.has_nulls && options.null_handling == MinMaxOptions::EMIT_NULL) { this->state = local; - return; + return Status::OK(); } if (local.has_nulls) { @@ -239,14 +243,16 @@ struct MinMaxImpl : public ScalarAggregator { } } this->state = local; + return Status::OK(); } - void MergeFrom(KernelContext*, KernelState&& src) override { + Status MergeFrom(KernelContext*, KernelState&& src) override { const auto& other = checked_cast(src); this->state += other.state; + return Status::OK(); } - void Finalize(KernelContext*, Datum* out) override { + Status Finalize(KernelContext*, Datum* out) override { using ScalarType = typename TypeTraits::ScalarType; std::vector> values; @@ -259,6 +265,7 @@ struct MinMaxImpl : public ScalarAggregator { std::make_shared(state.max)}; } out->value = std::make_shared(std::move(values), this->out_type); + return Status::OK(); } std::shared_ptr out_type; @@ -331,7 +338,7 @@ struct BooleanMinMaxImpl : public MinMaxImpl { using MinMaxImpl::MinMaxImpl; using MinMaxImpl::options; - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { StateType local; ArrayType arr(batch[0].array()); @@ -343,7 +350,7 @@ struct BooleanMinMaxImpl : public MinMaxImpl { local.has_values = valid_count > 0; if (local.has_nulls && options.null_handling == MinMaxOptions::EMIT_NULL) { this->state = local; - return; + return Status::OK(); } const auto true_count = arr.true_count(); @@ -352,6 +359,7 @@ struct BooleanMinMaxImpl : public MinMaxImpl { local.min = false_count == 0; this->state = local; + return Status::OK(); } }; @@ -386,8 +394,8 @@ struct MinMaxInitState { return Status::OK(); } - std::unique_ptr Create() { - ctx->SetStatus(VisitTypeInline(in_type, this)); + Result> Create() { + RETURN_NOT_OK(VisitTypeInline(in_type, this)); return std::move(state); } }; diff --git a/cpp/src/arrow/compute/kernels/aggregate_internal.h b/cpp/src/arrow/compute/kernels/aggregate_internal.h index 67337f22c5b..d74881108ae 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_internal.h +++ b/cpp/src/arrow/compute/kernels/aggregate_internal.h @@ -50,9 +50,9 @@ struct FindAccumulatorType> { }; struct ScalarAggregator : public KernelState { - virtual void Consume(KernelContext* ctx, const ExecBatch& batch) = 0; - virtual void MergeFrom(KernelContext* ctx, KernelState&& src) = 0; - virtual void Finalize(KernelContext* ctx, Datum* out) = 0; + virtual Status Consume(KernelContext* ctx, const ExecBatch& batch) = 0; + virtual Status MergeFrom(KernelContext* ctx, KernelState&& src) = 0; + virtual Status Finalize(KernelContext* ctx, Datum* out) = 0; }; void AddAggKernel(std::shared_ptr sig, KernelInit init, diff --git a/cpp/src/arrow/compute/kernels/aggregate_mode.cc b/cpp/src/arrow/compute/kernels/aggregate_mode.cc index 7ac0dd3c707..f7538ac5249 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_mode.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_mode.cc @@ -71,7 +71,7 @@ Result> PrepareOutput(int64_t n, KernelContext* ctx, // find top-n value:count pairs with minimal heap // suboptimal for tiny or large n, possibly okay as we're not in hot path template -void Finalize(KernelContext* ctx, Datum* out, Generator&& gen) { +Status Finalize(KernelContext* ctx, Datum* out, Generator&& gen) { using CType = typename InType::c_type; using ValueCountPair = std::pair; @@ -100,13 +100,15 @@ void Finalize(KernelContext* ctx, Datum* out, Generator&& gen) { CType* mode_buffer; int64_t* count_buffer; - KERNEL_ASSIGN_OR_RAISE(std::tie(mode_buffer, count_buffer), ctx, - PrepareOutput(n, ctx, out)); + ARROW_ASSIGN_OR_RAISE(std::tie(mode_buffer, count_buffer), + PrepareOutput(n, ctx, out)); for (int64_t i = n - 1; i >= 0; --i) { std::tie(mode_buffer[i], count_buffer[i]) = min_heap.top(); min_heap.pop(); } + + return Status::OK(); } // count value occurances for integers with narrow value range @@ -125,7 +127,7 @@ struct CountModer { this->counts.resize(value_range, 0); } - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // count values in all chunks, ignore nulls const Datum& datum = batch[0]; CountValues(this->counts.data(), datum, this->min); @@ -144,14 +146,14 @@ struct CountModer { return std::pair(0, kCountEOF); }; - Finalize(ctx, out, std::move(gen)); + return Finalize(ctx, out, std::move(gen)); } }; // booleans can be handled more straightforward template <> struct CountModer { - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { int64_t counts[2]{}; const Datum& datum = batch[0]; @@ -171,8 +173,8 @@ struct CountModer { bool* mode_buffer; int64_t* count_buffer; - KERNEL_ASSIGN_OR_RAISE(std::tie(mode_buffer, count_buffer), ctx, - PrepareOutput(n, ctx, out)); + ARROW_ASSIGN_OR_RAISE(std::tie(mode_buffer, count_buffer), + PrepareOutput(n, ctx, out)); if (n >= 1) { const bool index = counts[1] > counts[0]; @@ -183,6 +185,8 @@ struct CountModer { count_buffer[1] = counts[!index]; } } + + return Status::OK(); } }; @@ -193,7 +197,7 @@ struct SortModer { using CType = typename T::c_type; using Allocator = arrow::stl::allocator; - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // copy all chunks to a buffer, ignore nulls and nans std::vector in_buffer(Allocator(ctx->memory_pool())); @@ -238,7 +242,7 @@ struct SortModer { return std::make_pair(value, count); }; - Finalize(ctx, out, std::move(gen)); + return Finalize(ctx, out, std::move(gen)); } }; @@ -247,7 +251,7 @@ template struct CountOrSortModer { using CType = typename T::c_type; - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // cross point to benefit from counting approach // about 2x improvement for int32/64 from micro-benchmarking static constexpr int kMinArraySize = 8192; @@ -259,12 +263,11 @@ struct CountOrSortModer { std::tie(min, max) = GetMinMax(datum); if (static_cast(max) - static_cast(min) <= kMaxValueRange) { - CountModer(min, max).Exec(ctx, batch, out); - return; + return CountModer(min, max).Exec(ctx, batch, out); } } - SortModer().Exec(ctx, batch, out); + return SortModer().Exec(ctx, batch, out); } }; @@ -301,18 +304,16 @@ struct Moder::value>> { template struct ModeExecutor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (ctx->state() == nullptr) { - ctx->SetStatus(Status::Invalid("Mode requires ModeOptions")); - return; + return Status::Invalid("Mode requires ModeOptions"); } const ModeOptions& options = ModeState::Get(ctx); if (options.n <= 0) { - ctx->SetStatus(Status::Invalid("ModeOption::n must be strictly positive")); - return; + return Status::Invalid("ModeOption::n must be strictly positive"); } - Moder().impl.Exec(ctx, batch, out); + return Moder().impl.Exec(ctx, batch, out); } }; diff --git a/cpp/src/arrow/compute/kernels/aggregate_quantile.cc b/cpp/src/arrow/compute/kernels/aggregate_quantile.cc index f0de1be2793..2bb026dbdbd 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_quantile.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_quantile.cc @@ -77,7 +77,7 @@ struct SortQuantiler { using CType = typename InType::c_type; using Allocator = arrow::stl::allocator; - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const QuantileOptions& options = QuantileState::Get(ctx); // copy all chunks to a buffer, ignore nulls and nans @@ -111,8 +111,8 @@ struct SortQuantiler { // calculate quantiles if (out_length > 0) { - KERNEL_ASSIGN_OR_RAISE(out_data->buffers[1], ctx, - ctx->Allocate(out_length * GetBitWidth(*out_type) / 8)); + ARROW_ASSIGN_OR_RAISE(out_data->buffers[1], + ctx->Allocate(out_length * GetBitWidth(*out_type) / 8)); // find quantiles in descending order std::vector q_indices(out_length); @@ -143,6 +143,7 @@ struct SortQuantiler { } *out = Datum(std::move(out_data)); + return Status::OK(); } // return quantile located exactly at some input data point @@ -226,7 +227,7 @@ struct CountQuantiler { this->counts.resize(value_range, 0); } - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const QuantileOptions& options = QuantileState::Get(ctx); // count values in all chunks, ignore nulls @@ -247,8 +248,8 @@ struct CountQuantiler { // calculate quantiles if (out_length > 0) { - KERNEL_ASSIGN_OR_RAISE(out_data->buffers[1], ctx, - ctx->Allocate(out_length * GetBitWidth(*out_type) / 8)); + ARROW_ASSIGN_OR_RAISE(out_data->buffers[1], + ctx->Allocate(out_length * GetBitWidth(*out_type) / 8)); // find quantiles in ascending order std::vector q_indices(out_length); @@ -277,6 +278,7 @@ struct CountQuantiler { } *out = Datum(std::move(out_data)); + return Status::OK(); } // return quantile located exactly at some input data point @@ -341,7 +343,7 @@ template struct CountOrSortQuantiler { using CType = typename InType::c_type; - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // cross point to benefit from histogram approach // parameters estimated from ad-hoc benchmarks manually static constexpr int kMinArraySize = 65536; @@ -353,12 +355,11 @@ struct CountOrSortQuantiler { std::tie(min, max) = GetMinMax(datum); if (static_cast(max) - static_cast(min) <= kMaxValueRange) { - CountQuantiler(min, max).Exec(ctx, batch, out); - return; + return CountQuantiler(min, max).Exec(ctx, batch, out); } } - SortQuantiler().Exec(ctx, batch, out); + return SortQuantiler().Exec(ctx, batch, out); } }; @@ -390,25 +391,22 @@ struct ExactQuantiler::value>> { template struct QuantileExecutor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (ctx->state() == nullptr) { - ctx->SetStatus(Status::Invalid("Quantile requires QuantileOptions")); - return; + return Status::Invalid("Quantile requires QuantileOptions"); } const QuantileOptions& options = QuantileState::Get(ctx); if (options.q.empty()) { - ctx->SetStatus(Status::Invalid("Requires quantile argument")); - return; + return Status::Invalid("Requires quantile argument"); } for (double q : options.q) { if (q < 0 || q > 1) { - ctx->SetStatus(Status::Invalid("Quantile must be between 0 and 1")); - return; + return Status::Invalid("Quantile must be between 0 and 1"); } } - ExactQuantiler().impl.Exec(ctx, batch, out); + return ExactQuantiler().impl.Exec(ctx, batch, out); } }; diff --git a/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc b/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc index fc8f43b0ae2..fb474a6b8b3 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_tdigest.cc @@ -39,7 +39,7 @@ struct TDigestImpl : public ScalarAggregator { explicit TDigestImpl(const TDigestOptions& options) : q{options.q}, tdigest{options.delta, options.buffer_size} {} - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { const ArrayData& data = *batch[0].array(); const CType* values = data.GetValues(1); @@ -51,23 +51,25 @@ struct TDigestImpl : public ScalarAggregator { } }); } + return Status::OK(); } - void MergeFrom(KernelContext*, KernelState&& src) override { + Status MergeFrom(KernelContext*, KernelState&& src) override { auto& other = checked_cast(src); std::vector other_tdigest; other_tdigest.push_back(std::move(other.tdigest)); this->tdigest.Merge(&other_tdigest); + return Status::OK(); } - void Finalize(KernelContext* ctx, Datum* out) override { + Status Finalize(KernelContext* ctx, Datum* out) override { const int64_t out_length = this->tdigest.is_empty() ? 0 : this->q.size(); auto out_data = ArrayData::Make(float64(), out_length, 0); out_data->buffers.resize(2, nullptr); if (out_length > 0) { - KERNEL_ASSIGN_OR_RAISE(out_data->buffers[1], ctx, - ctx->Allocate(out_length * sizeof(double))); + ARROW_ASSIGN_OR_RAISE(out_data->buffers[1], + ctx->Allocate(out_length * sizeof(double))); double* out_buffer = out_data->template GetMutableValues(1); for (int64_t i = 0; i < out_length; ++i) { out_buffer[i] = this->tdigest.Quantile(this->q[i]); @@ -75,6 +77,7 @@ struct TDigestImpl : public ScalarAggregator { } *out = Datum(std::move(out_data)); + return Status::OK(); } const std::vector& q; @@ -105,13 +108,14 @@ struct TDigestInitState { return Status::OK(); } - std::unique_ptr Create() { - ctx->SetStatus(VisitTypeInline(in_type, this)); + Result> Create() { + RETURN_NOT_OK(VisitTypeInline(in_type, this)); return std::move(state); } }; -std::unique_ptr TDigestInit(KernelContext* ctx, const KernelInitArgs& args) { +Result> TDigestInit(KernelContext* ctx, + const KernelInitArgs& args) { TDigestInitState visitor(ctx, *args.inputs[0].type, static_cast(*args.options)); return visitor.Create(); diff --git a/cpp/src/arrow/compute/kernels/aggregate_var_std.cc b/cpp/src/arrow/compute/kernels/aggregate_var_std.cc index d11e73efd77..29b2adce3bd 100644 --- a/cpp/src/arrow/compute/kernels/aggregate_var_std.cc +++ b/cpp/src/arrow/compute/kernels/aggregate_var_std.cc @@ -151,17 +151,19 @@ struct VarStdImpl : public ScalarAggregator { const VarianceOptions& options, VarOrStd return_type) : out_type(out_type), options(options), return_type(return_type) {} - void Consume(KernelContext*, const ExecBatch& batch) override { + Status Consume(KernelContext*, const ExecBatch& batch) override { ArrayType array(batch[0].array()); this->state.Consume(array); + return Status::OK(); } - void MergeFrom(KernelContext*, KernelState&& src) override { + Status MergeFrom(KernelContext*, KernelState&& src) override { const auto& other = checked_cast(src); this->state.MergeFrom(other.state); + return Status::OK(); } - void Finalize(KernelContext*, Datum* out) override { + Status Finalize(KernelContext*, Datum* out) override { if (this->state.count <= options.ddof) { out->value = std::make_shared(); } else { @@ -169,6 +171,7 @@ struct VarStdImpl : public ScalarAggregator { out->value = std::make_shared(return_type == VarOrStd::Var ? var : sqrt(var)); } + return Status::OK(); } std::shared_ptr out_type; @@ -208,21 +211,22 @@ struct VarStdInitState { return Status::OK(); } - std::unique_ptr Create() { - ctx->SetStatus(VisitTypeInline(in_type, this)); + Result> Create() { + RETURN_NOT_OK(VisitTypeInline(in_type, this)); return std::move(state); } }; -std::unique_ptr StddevInit(KernelContext* ctx, const KernelInitArgs& args) { +Result> StddevInit(KernelContext* ctx, + const KernelInitArgs& args) { VarStdInitState visitor( ctx, *args.inputs[0].type, args.kernel->signature->out_type().type(), static_cast(*args.options), VarOrStd::Std); return visitor.Create(); } -std::unique_ptr VarianceInit(KernelContext* ctx, - const KernelInitArgs& args) { +Result> VarianceInit(KernelContext* ctx, + const KernelInitArgs& args) { VarStdInitState visitor( ctx, *args.inputs[0].type, args.kernel->signature->out_type().type(), static_cast(*args.options), VarOrStd::Var); diff --git a/cpp/src/arrow/compute/kernels/codegen_internal.cc b/cpp/src/arrow/compute/kernels/codegen_internal.cc index ad43b7a3aa9..d6a1d4ccbc4 100644 --- a/cpp/src/arrow/compute/kernels/codegen_internal.cc +++ b/cpp/src/arrow/compute/kernels/codegen_internal.cc @@ -28,15 +28,15 @@ namespace arrow { namespace compute { namespace internal { -void ExecFail(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - ctx->SetStatus(Status::NotImplemented("This kernel is malformed")); +Status ExecFail(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return Status::NotImplemented("This kernel is malformed"); } ArrayKernelExec MakeFlippedBinaryExec(ArrayKernelExec exec) { return [exec](KernelContext* ctx, const ExecBatch& batch, Datum* out) { ExecBatch flipped_batch = batch; std::swap(flipped_batch.values[0], flipped_batch.values[1]); - exec(ctx, flipped_batch, out); + return exec(ctx, flipped_batch, out); }; } diff --git a/cpp/src/arrow/compute/kernels/codegen_internal.h b/cpp/src/arrow/compute/kernels/codegen_internal.h index b5d6c3807f1..7b394565f7d 100644 --- a/cpp/src/arrow/compute/kernels/codegen_internal.h +++ b/cpp/src/arrow/compute/kernels/codegen_internal.h @@ -63,42 +63,6 @@ using internal::VisitTwoBitBlocksVoid; namespace compute { namespace internal { -#ifdef ARROW_EXTRA_ERROR_CONTEXT - -#define KERNEL_RETURN_IF_ERROR(ctx, expr) \ - do { \ - Status _st = (expr); \ - if (ARROW_PREDICT_FALSE(!_st.ok())) { \ - _st.AddContextLine(__FILE__, __LINE__, #expr); \ - ctx->SetStatus(_st); \ - return; \ - } \ - } while (0) - -#else - -#define KERNEL_RETURN_IF_ERROR(ctx, expr) \ - do { \ - Status _st = (expr); \ - if (ARROW_PREDICT_FALSE(!_st.ok())) { \ - ctx->SetStatus(_st); \ - return; \ - } \ - } while (0) - -#endif // ARROW_EXTRA_ERROR_CONTEXT - -#define KERNEL_ASSIGN_OR_RAISE_IMPL(result_name, lhs, ctx, rexpr) \ - auto result_name = (rexpr); \ - KERNEL_RETURN_IF_ERROR(ctx, (result_name).status()); \ - lhs = std::move(result_name).MoveValueUnsafe(); - -#define KERNEL_ASSIGN_OR_RAISE_NAME(x, y) ARROW_CONCAT(x, y) - -#define KERNEL_ASSIGN_OR_RAISE(lhs, ctx, rexpr) \ - KERNEL_ASSIGN_OR_RAISE_IMPL(KERNEL_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \ - lhs, ctx, rexpr); - /// KernelState adapter for the common case of kernels whose only /// state is an instance of a subclass of FunctionOptions. /// Default FunctionOptions are *not* handled here. @@ -106,15 +70,14 @@ template struct OptionsWrapper : public KernelState { explicit OptionsWrapper(OptionsType options) : options(std::move(options)) {} - static std::unique_ptr Init(KernelContext* ctx, - const KernelInitArgs& args) { + static Result> Init(KernelContext* ctx, + const KernelInitArgs& args) { if (auto options = static_cast(args.options)) { return ::arrow::internal::make_unique(*options); } - ctx->SetStatus( - Status::Invalid("Attempted to initialize KernelState from null FunctionOptions")); - return NULLPTR; + return Status::Invalid( + "Attempted to initialize KernelState from null FunctionOptions"); } static const OptionsType& Get(const KernelState& state) { @@ -133,16 +96,15 @@ struct KernelStateFromFunctionOptions : public KernelState { explicit KernelStateFromFunctionOptions(KernelContext* ctx, OptionsType state) : state(StateType(ctx, std::move(state))) {} - static std::unique_ptr Init(KernelContext* ctx, - const KernelInitArgs& args) { + static Result> Init(KernelContext* ctx, + const KernelInitArgs& args) { if (auto options = static_cast(args.options)) { return ::arrow::internal::make_unique(ctx, *options); } - ctx->SetStatus( - Status::Invalid("Attempted to initialize KernelState from null FunctionOptions")); - return NULLPTR; + return Status::Invalid( + "Attempted to initialize KernelState from null FunctionOptions"); } static const StateType& Get(const KernelState& state) { @@ -372,8 +334,9 @@ struct BoxScalar { // values, such as Decimal128 rather than util::string_view. template -static void VisitArrayValuesInline(const ArrayData& arr, VisitFunc&& valid_func, - NullFunc&& null_func) { +static typename arrow::internal::call_traits::enable_if_return::type +VisitArrayValuesInline(const ArrayData& arr, VisitFunc&& valid_func, + NullFunc&& null_func) { VisitArrayDataInline( arr, [&](typename GetViewType::PhysicalType v) { @@ -382,6 +345,18 @@ static void VisitArrayValuesInline(const ArrayData& arr, VisitFunc&& valid_func, std::forward(null_func)); } +template +static typename arrow::internal::call_traits::enable_if_return::type +VisitArrayValuesInline(const ArrayData& arr, VisitFunc&& valid_func, + NullFunc&& null_func) { + VisitArrayDataInline( + arr, + [&](typename GetViewType::PhysicalType v) { + return valid_func(GetViewType::LogicalValue(std::move(v))); + }, + std::forward(null_func)); +} + // Like VisitArrayValuesInline, but for binary functions. template @@ -411,7 +386,7 @@ Result FirstType(KernelContext*, const std::vector& desc // ---------------------------------------------------------------------- // Generate an array kernel given template classes -void ExecFail(KernelContext* ctx, const ExecBatch& batch, Datum* out); +Status ExecFail(KernelContext* ctx, const ExecBatch& batch, Datum* out); ArrayKernelExec MakeFlippedBinaryExec(ArrayKernelExec exec); @@ -469,15 +444,16 @@ namespace applicator { // // Operator must implement // -// static void Call(KernelContext*, const ArrayData& in, ArrayData* out) -// static void Call(KernelContext*, const Scalar& in, Scalar* out) +// static Status Call(KernelContext*, const ArrayData& in, ArrayData* out) +// static Status Call(KernelContext*, const Scalar& in, Scalar* out) template -static void SimpleUnary(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +static Status SimpleUnary(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (batch[0].kind() == Datum::SCALAR) { - Operator::Call(ctx, *batch[0].scalar(), out->scalar().get()); + return Operator::Call(ctx, *batch[0].scalar(), out->scalar().get()); } else if (batch.length > 0) { - Operator::Call(ctx, *batch[0].array(), out->mutable_array()); + return Operator::Call(ctx, *batch[0].array(), out->mutable_array()); } + return Status::OK(); } // Generate an ArrayKernelExec given a functor that handles all of its own @@ -485,29 +461,33 @@ static void SimpleUnary(KernelContext* ctx, const ExecBatch& batch, Datum* out) // // Operator must implement // -// static void Call(KernelContext*, const ArrayData& arg0, const ArrayData& arg1, -// ArrayData* out) -// static void Call(KernelContext*, const ArrayData& arg0, const Scalar& arg1, -// ArrayData* out) -// static void Call(KernelContext*, const Scalar& arg0, const ArrayData& arg1, -// ArrayData* out) -// static void Call(KernelContext*, const Scalar& arg0, const Scalar& arg1, -// Scalar* out) +// static Status Call(KernelContext*, const ArrayData& arg0, const ArrayData& arg1, +// ArrayData* out) +// static Status Call(KernelContext*, const ArrayData& arg0, const Scalar& arg1, +// ArrayData* out) +// static Status Call(KernelContext*, const Scalar& arg0, const ArrayData& arg1, +// ArrayData* out) +// static Status Call(KernelContext*, const Scalar& arg0, const Scalar& arg1, +// Scalar* out) template -static void SimpleBinary(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - if (batch.length == 0) return; +static Status SimpleBinary(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + if (batch.length == 0) return Status::OK(); if (batch[0].kind() == Datum::ARRAY) { if (batch[1].kind() == Datum::ARRAY) { - Operator::Call(ctx, *batch[0].array(), *batch[1].array(), out->mutable_array()); + return Operator::Call(ctx, *batch[0].array(), *batch[1].array(), + out->mutable_array()); } else { - Operator::Call(ctx, *batch[0].array(), *batch[1].scalar(), out->mutable_array()); + return Operator::Call(ctx, *batch[0].array(), *batch[1].scalar(), + out->mutable_array()); } } else { if (batch[1].kind() == Datum::ARRAY) { - Operator::Call(ctx, *batch[0].scalar(), *batch[1].array(), out->mutable_array()); + return Operator::Call(ctx, *batch[0].scalar(), *batch[1].array(), + out->mutable_array()); } else { - Operator::Call(ctx, *batch[0].scalar(), *batch[1].scalar(), out->scalar().get()); + return Operator::Call(ctx, *batch[0].scalar(), *batch[1].scalar(), + out->scalar().get()); } } } @@ -522,32 +502,34 @@ struct OutputAdapter; template struct OutputAdapter> { template - static void Write(KernelContext*, Datum* out, Generator&& generator) { + static Status Write(KernelContext*, Datum* out, Generator&& generator) { ArrayData* out_arr = out->mutable_array(); auto out_bitmap = out_arr->buffers[1]->mutable_data(); GenerateBitsUnrolled(out_bitmap, out_arr->offset, out_arr->length, std::forward(generator)); + return Status::OK(); } }; template struct OutputAdapter> { template - static void Write(KernelContext*, Datum* out, Generator&& generator) { + static Status Write(KernelContext*, Datum* out, Generator&& generator) { ArrayData* out_arr = out->mutable_array(); auto out_data = out_arr->GetMutableValues(1); // TODO: Is this as fast as a more explicitly inlined function? for (int64_t i = 0; i < out_arr->length; ++i) { *out_data++ = generator(); } + return Status::OK(); } }; template struct OutputAdapter> { template - static void Write(KernelContext* ctx, Datum* out, Generator&& generator) { - ctx->SetStatus(Status::NotImplemented("NYI")); + static Status Write(KernelContext* ctx, Datum* out, Generator&& generator) { + return Status::NotImplemented("NYI"); } }; @@ -563,8 +545,10 @@ struct OutputAdapter> { // // struct Op { // template -// static OutValue Call(KernelContext* ctx, Arg0Value val) { +// static OutValue Call(KernelContext* ctx, Arg0Value val, Status* st) { // // implementation +// // NOTE: "status" should only populated with errors, +// // leave it unmodified to indicate Status::OK() // } // }; template @@ -572,26 +556,30 @@ struct ScalarUnary { using OutValue = typename GetOutputType::T; using Arg0Value = typename GetViewType::T; - static void ExecArray(KernelContext* ctx, const ArrayData& arg0, Datum* out) { + static Status ExecArray(KernelContext* ctx, const ArrayData& arg0, Datum* out) { + Status st = Status::OK(); ArrayIterator arg0_it(arg0); - OutputAdapter::Write(ctx, out, [&]() -> OutValue { - return Op::template Call(ctx, arg0_it()); - }); + RETURN_NOT_OK(OutputAdapter::Write(ctx, out, [&]() -> OutValue { + return Op::template Call(ctx, arg0_it(), &st); + })); + return st; } - static void ExecScalar(KernelContext* ctx, const Scalar& arg0, Datum* out) { + static Status ExecScalar(KernelContext* ctx, const Scalar& arg0, Datum* out) { + Status st = Status::OK(); Scalar* out_scalar = out->scalar().get(); if (arg0.is_valid) { Arg0Value arg0_val = UnboxScalar::Unbox(arg0); out_scalar->is_valid = true; - BoxScalar::Box(Op::template Call(ctx, arg0_val), + BoxScalar::Box(Op::template Call(ctx, arg0_val, &st), out_scalar); } else { out_scalar->is_valid = false; } + return st; } - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (batch[0].kind() == Datum::ARRAY) { return ExecArray(ctx, *batch[0].array(), out); } else { @@ -615,66 +603,69 @@ struct ScalarUnaryNotNullStateful { template struct ArrayExec { - static void Exec(const ThisType& functor, KernelContext* ctx, const ExecBatch& batch, - Datum* out) { + static Status Exec(const ThisType& functor, KernelContext* ctx, + const ExecBatch& batch, Datum* out) { ARROW_LOG(FATAL) << "Missing ArrayExec specialization for output type " << out->type(); + return Status::NotImplemented("NYI"); } }; template struct ArrayExec< Type, enable_if_t::value && !is_boolean_type::value>> { - static void Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, - Datum* out) { + static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, + Datum* out) { + Status st = Status::OK(); ArrayData* out_arr = out->mutable_array(); auto out_data = out_arr->GetMutableValues(1); VisitArrayValuesInline( arg0, [&](Arg0Value v) { - *out_data++ = functor.op.template Call(ctx, v); + *out_data++ = functor.op.template Call(ctx, v, &st); }, [&]() { // null ++out_data; }); + return st; } }; template struct ArrayExec> { - static void Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, - Datum* out) { + static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, + Datum* out) { // NOTE: This code is not currently used by any kernels and has // suboptimal performance because it's recomputing the validity bitmap // that is already computed by the kernel execution layer. Consider // writing a lower-level "output adapter" for base binary types. typename TypeTraits::BuilderType builder; - VisitArrayValuesInline( - arg0, - [&](Arg0Value v) { - KERNEL_RETURN_IF_ERROR(ctx, builder.Append(functor.op.Call(ctx, v))); - }, - [&]() { KERNEL_RETURN_IF_ERROR(ctx, builder.AppendNull()); }); - if (!ctx->HasError()) { + Status st = Status::OK(); + RETURN_NOT_OK(VisitArrayValuesInline( + arg0, [&](Arg0Value v) { return builder.Append(functor.op.Call(ctx, v, &st)); }, + [&]() { return builder.AppendNull(); })); + if (st.ok()) { std::shared_ptr result; - ctx->SetStatus(builder.FinishInternal(&result)); + RETURN_NOT_OK(builder.FinishInternal(&result)); out->value = std::move(result); } + return st; } }; template struct ArrayExec::value>> { - static void Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, - Datum* out) { + static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, + Datum* out) { + Status st = Status::OK(); ArrayData* out_arr = out->mutable_array(); FirstTimeBitmapWriter out_writer(out_arr->buffers[1]->mutable_data(), out_arr->offset, out_arr->length); VisitArrayValuesInline( arg0, [&](Arg0Value v) { - if (functor.op.template Call(ctx, v)) { + if (functor.op.template Call(ctx, v, &st)) { out_writer.Set(); } out_writer.Next(); @@ -685,13 +676,15 @@ struct ScalarUnaryNotNullStateful { out_writer.Next(); }); out_writer.Finish(); + return st; } }; template struct ArrayExec> { - static void Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, - Datum* out) { + static Status Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0, + Datum* out) { + Status st = Status::OK(); ArrayData* out_arr = out->mutable_array(); // Decimal128 data buffers are not safely reinterpret_cast-able on big-endian using endian_agnostic = @@ -700,24 +693,28 @@ struct ScalarUnaryNotNullStateful { VisitArrayValuesInline( arg0, [&](Arg0Value v) { - functor.op.template Call(ctx, v).ToBytes( - out_data++->data()); + functor.op.template Call(ctx, v, &st) + .ToBytes(out_data++->data()); }, [&]() { ++out_data; }); + return st; } }; - void Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) { + Status Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) { + Status st = Status::OK(); if (arg0.is_valid) { Arg0Value arg0_val = UnboxScalar::Unbox(arg0); - BoxScalar::Box(this->op.template Call(ctx, arg0_val), - out->scalar().get()); + BoxScalar::Box( + this->op.template Call(ctx, arg0_val, &st), + out->scalar().get()); } + return st; } - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (batch[0].kind() == Datum::ARRAY) { - ArrayExec::Exec(*this, ctx, *batch[0].array(), out); + return ArrayExec::Exec(*this, ctx, *batch[0].array(), out); } else { return Scalar(ctx, *batch[0].scalar(), out); } @@ -732,7 +729,7 @@ struct ScalarUnaryNotNull { using OutValue = typename GetOutputType::T; using Arg0Value = typename GetViewType::T; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // Seed kernel with dummy state ScalarUnaryNotNullStateful kernel({}); return kernel.Exec(ctx, batch, out); @@ -751,8 +748,11 @@ struct ScalarUnaryNotNull { // // struct Op { // template -// static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1) { +// static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1, Status* st) +// { // // implementation +// // NOTE: "status" should only populated with errors, +// // leave it unmodified to indicate Status::OK() // } // }; template @@ -761,44 +761,52 @@ struct ScalarBinary { using Arg0Value = typename GetViewType::T; using Arg1Value = typename GetViewType::T; - static void ArrayArray(KernelContext* ctx, const ArrayData& arg0, const ArrayData& arg1, - Datum* out) { + static Status ArrayArray(KernelContext* ctx, const ArrayData& arg0, + const ArrayData& arg1, Datum* out) { + Status st = Status::OK(); ArrayIterator arg0_it(arg0); ArrayIterator arg1_it(arg1); - OutputAdapter::Write(ctx, out, [&]() -> OutValue { - return Op::template Call(ctx, arg0_it(), arg1_it()); - }); + RETURN_NOT_OK(OutputAdapter::Write(ctx, out, [&]() -> OutValue { + return Op::template Call(ctx, arg0_it(), arg1_it(), &st); + })); + return st; } - static void ArrayScalar(KernelContext* ctx, const ArrayData& arg0, const Scalar& arg1, - Datum* out) { + static Status ArrayScalar(KernelContext* ctx, const ArrayData& arg0, const Scalar& arg1, + Datum* out) { + Status st = Status::OK(); ArrayIterator arg0_it(arg0); auto arg1_val = UnboxScalar::Unbox(arg1); - OutputAdapter::Write(ctx, out, [&]() -> OutValue { - return Op::template Call(ctx, arg0_it(), arg1_val); - }); + RETURN_NOT_OK(OutputAdapter::Write(ctx, out, [&]() -> OutValue { + return Op::template Call(ctx, arg0_it(), arg1_val, &st); + })); + return st; } - static void ScalarArray(KernelContext* ctx, const Scalar& arg0, const ArrayData& arg1, - Datum* out) { + static Status ScalarArray(KernelContext* ctx, const Scalar& arg0, const ArrayData& arg1, + Datum* out) { + Status st = Status::OK(); auto arg0_val = UnboxScalar::Unbox(arg0); ArrayIterator arg1_it(arg1); - OutputAdapter::Write(ctx, out, [&]() -> OutValue { - return Op::template Call(ctx, arg0_val, arg1_it()); - }); + RETURN_NOT_OK(OutputAdapter::Write(ctx, out, [&]() -> OutValue { + return Op::template Call(ctx, arg0_val, arg1_it(), &st); + })); + return st; } - static void ScalarScalar(KernelContext* ctx, const Scalar& arg0, const Scalar& arg1, - Datum* out) { + static Status ScalarScalar(KernelContext* ctx, const Scalar& arg0, const Scalar& arg1, + Datum* out) { + Status st = Status::OK(); if (out->scalar()->is_valid) { auto arg0_val = UnboxScalar::Unbox(arg0); auto arg1_val = UnboxScalar::Unbox(arg1); - BoxScalar::Box(Op::template Call(ctx, arg0_val, arg1_val), + BoxScalar::Box(Op::template Call(ctx, arg0_val, arg1_val, &st), out->scalar().get()); } + return st; } - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (batch[0].kind() == Datum::ARRAY) { if (batch[1].kind() == Datum::ARRAY) { return ArrayArray(ctx, *batch[0].array(), *batch[1].array(), out); @@ -829,19 +837,22 @@ struct ScalarBinaryNotNullStateful { // NOTE: In ArrayExec, Type is really OutputType - void ArrayArray(KernelContext* ctx, const ArrayData& arg0, const ArrayData& arg1, - Datum* out) { + Status ArrayArray(KernelContext* ctx, const ArrayData& arg0, const ArrayData& arg1, + Datum* out) { + Status st = Status::OK(); OutputArrayWriter writer(out->mutable_array()); VisitTwoArrayValuesInline( arg0, arg1, [&](Arg0Value u, Arg1Value v) { - writer.Write(op.template Call(ctx, u, v)); + writer.Write(op.template Call(ctx, u, v, &st)); }, [&]() { writer.WriteNull(); }); + return st; } - void ArrayScalar(KernelContext* ctx, const ArrayData& arg0, const Scalar& arg1, - Datum* out) { + Status ArrayScalar(KernelContext* ctx, const ArrayData& arg0, const Scalar& arg1, + Datum* out) { + Status st = Status::OK(); OutputArrayWriter writer(out->mutable_array()); if (arg1.is_valid) { const auto arg1_val = UnboxScalar::Unbox(arg1); @@ -849,14 +860,16 @@ struct ScalarBinaryNotNullStateful { arg0, [&](Arg0Value u) { writer.Write( - op.template Call(ctx, u, arg1_val)); + op.template Call(ctx, u, arg1_val, &st)); }, [&]() { writer.WriteNull(); }); } + return st; } - void ScalarArray(KernelContext* ctx, const Scalar& arg0, const ArrayData& arg1, - Datum* out) { + Status ScalarArray(KernelContext* ctx, const Scalar& arg0, const ArrayData& arg1, + Datum* out) { + Status st = Status::OK(); OutputArrayWriter writer(out->mutable_array()); if (arg0.is_valid) { const auto arg0_val = UnboxScalar::Unbox(arg0); @@ -864,24 +877,27 @@ struct ScalarBinaryNotNullStateful { arg1, [&](Arg1Value v) { writer.Write( - op.template Call(ctx, arg0_val, v)); + op.template Call(ctx, arg0_val, v, &st)); }, [&]() { writer.WriteNull(); }); } + return st; } - void ScalarScalar(KernelContext* ctx, const Scalar& arg0, const Scalar& arg1, - Datum* out) { + Status ScalarScalar(KernelContext* ctx, const Scalar& arg0, const Scalar& arg1, + Datum* out) { + Status st = Status::OK(); if (arg0.is_valid && arg1.is_valid) { const auto arg0_val = UnboxScalar::Unbox(arg0); const auto arg1_val = UnboxScalar::Unbox(arg1); BoxScalar::Box( - op.template Call(ctx, arg0_val, arg1_val), + op.template Call(ctx, arg0_val, arg1_val, &st), out->scalar().get()); } + return st; } - void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (batch[0].kind() == Datum::ARRAY) { if (batch[1].kind() == Datum::ARRAY) { return ArrayArray(ctx, *batch[0].array(), *batch[1].array(), out); @@ -908,7 +924,7 @@ struct ScalarBinaryNotNull { using Arg0Value = typename GetViewType::T; using Arg1Value = typename GetViewType::T; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // Seed kernel with dummy state ScalarBinaryNotNullStateful kernel({}); return kernel.Exec(ctx, batch, out); diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index dccbe50d9cf..f45e82e04af 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -770,12 +770,11 @@ HashAggregateKernel MakeKernel(InputType argument_type) { HashAggregateKernel kernel; kernel.init = [](KernelContext* ctx, - const KernelInitArgs& args) -> std::unique_ptr { + const KernelInitArgs& args) -> Result> { auto impl = ::arrow::internal::make_unique(); // FIXME(bkietz) Init should not take a type. That should be an unboxed template arg // for the Impl. Otherwise we're not exposing dispatch as well as we should. - ctx->SetStatus(impl->Init(ctx->exec_context(), args.options, args.inputs[0].type)); - if (ctx->HasError()) return nullptr; + RETURN_NOT_OK(impl->Init(ctx->exec_context(), args.options, args.inputs[0].type)); return std::move(impl); }; @@ -788,17 +787,18 @@ HashAggregateKernel MakeKernel(InputType argument_type) { })); kernel.consume = [](KernelContext* ctx, const ExecBatch& batch) { - ctx->SetStatus(checked_cast(ctx->state())->Consume(batch)); + return checked_cast(ctx->state())->Consume(batch); }; kernel.merge = [](KernelContext* ctx, KernelState&&, KernelState*) { // TODO(ARROW-11840) merge two hash tables - ctx->SetStatus(Status::NotImplemented("Merge hashed aggregations")); + return Status::NotImplemented("Merge hashed aggregations"); }; kernel.finalize = [](KernelContext* ctx, Datum* out) { - KERNEL_ASSIGN_OR_RAISE(*out, ctx, - checked_cast(ctx->state())->Finalize()); + ARROW_ASSIGN_OR_RAISE(*out, + checked_cast(ctx->state())->Finalize()); + return Status::OK(); }; return kernel; @@ -843,14 +843,14 @@ Result>> InitKernels( } KernelContext kernel_ctx{ctx}; - states[i] = kernels[i]->init(&kernel_ctx, KernelInitArgs{kernels[i], - { - in_descrs[i].type, - uint32(), - uint32(), - }, - options}); - if (kernel_ctx.HasError()) return kernel_ctx.status(); + ARROW_ASSIGN_OR_RAISE( + states[i], kernels[i]->init(&kernel_ctx, KernelInitArgs{kernels[i], + { + in_descrs[i].type, + uint32(), + uint32(), + }, + options})); } return std::move(states); @@ -936,8 +936,7 @@ Result GroupBy(const std::vector& arguments, const std::vectornum_groups())})); - kernels[i]->consume(&batch_ctx, batch); - if (batch_ctx.HasError()) return batch_ctx.status(); + RETURN_NOT_OK(kernels[i]->consume(&batch_ctx, batch)); } } @@ -949,8 +948,7 @@ Result GroupBy(const std::vector& arguments, const std::vectorfinalize(&batch_ctx, &out); - if (batch_ctx.HasError()) return batch_ctx.status(); + RETURN_NOT_OK(kernels[i]->finalize(&batch_ctx, &out)); *it++ = out.array(); } diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc index 260721b08d9..7b9b23e7ff8 100644 --- a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc +++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc @@ -64,34 +64,37 @@ constexpr Unsigned to_unsigned(T signed_) { struct Add { template - static constexpr enable_if_floating_point Call(KernelContext*, T left, T right) { + static constexpr enable_if_floating_point Call(KernelContext*, T left, T right, + Status*) { return left + right; } template - static constexpr enable_if_unsigned_integer Call(KernelContext*, T left, T right) { + static constexpr enable_if_unsigned_integer Call(KernelContext*, T left, T right, + Status*) { return left + right; } template - static constexpr enable_if_signed_integer Call(KernelContext*, T left, T right) { + static constexpr enable_if_signed_integer Call(KernelContext*, T left, T right, + Status*) { return arrow::internal::SafeSignedAdd(left, right); } }; struct AddChecked { template - enable_if_integer Call(KernelContext* ctx, Arg0 left, Arg1 right) { + enable_if_integer Call(KernelContext*, Arg0 left, Arg1 right, Status* st) { static_assert(std::is_same::value && std::is_same::value, ""); T result = 0; if (ARROW_PREDICT_FALSE(AddWithOverflow(left, right, &result))) { - ctx->SetStatus(Status::Invalid("overflow")); + *st = Status::Invalid("overflow"); } return result; } template - enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right) { + enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right, Status*) { static_assert(std::is_same::value && std::is_same::value, ""); return left + right; } @@ -99,34 +102,37 @@ struct AddChecked { struct Subtract { template - static constexpr enable_if_floating_point Call(KernelContext*, T left, T right) { + static constexpr enable_if_floating_point Call(KernelContext*, T left, T right, + Status*) { return left - right; } template - static constexpr enable_if_unsigned_integer Call(KernelContext*, T left, T right) { + static constexpr enable_if_unsigned_integer Call(KernelContext*, T left, T right, + Status*) { return left - right; } template - static constexpr enable_if_signed_integer Call(KernelContext*, T left, T right) { + static constexpr enable_if_signed_integer Call(KernelContext*, T left, T right, + Status*) { return arrow::internal::SafeSignedSubtract(left, right); } }; struct SubtractChecked { template - enable_if_integer Call(KernelContext* ctx, Arg0 left, Arg1 right) { + enable_if_integer Call(KernelContext*, Arg0 left, Arg1 right, Status* st) { static_assert(std::is_same::value && std::is_same::value, ""); T result = 0; if (ARROW_PREDICT_FALSE(SubtractWithOverflow(left, right, &result))) { - ctx->SetStatus(Status::Invalid("overflow")); + *st = Status::Invalid("overflow"); } return result; } template - enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right) { + enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right, Status*) { static_assert(std::is_same::value && std::is_same::value, ""); return left - right; } @@ -143,17 +149,20 @@ struct Multiply { static_assert(std::is_same::value, ""); template - static constexpr enable_if_floating_point Call(KernelContext*, T left, T right) { + static constexpr enable_if_floating_point Call(KernelContext*, T left, T right, + Status*) { return left * right; } template - static constexpr enable_if_unsigned_integer Call(KernelContext*, T left, T right) { + static constexpr enable_if_unsigned_integer Call(KernelContext*, T left, T right, + Status*) { return left * right; } template - static constexpr enable_if_signed_integer Call(KernelContext*, T left, T right) { + static constexpr enable_if_signed_integer Call(KernelContext*, T left, T right, + Status*) { return to_unsigned(left) * to_unsigned(right); } @@ -162,28 +171,28 @@ struct Multiply { // behaviour). Therefore we first cast to 32 bit unsigned integers where overflow is // well defined. template - static constexpr int16_t Call(KernelContext*, int16_t left, int16_t right) { + static constexpr int16_t Call(KernelContext*, int16_t left, int16_t right, Status*) { return static_cast(left) * static_cast(right); } template - static constexpr uint16_t Call(KernelContext*, uint16_t left, uint16_t right) { + static constexpr uint16_t Call(KernelContext*, uint16_t left, uint16_t right, Status*) { return static_cast(left) * static_cast(right); } }; struct MultiplyChecked { template - enable_if_integer Call(KernelContext* ctx, Arg0 left, Arg1 right) { + enable_if_integer Call(KernelContext*, Arg0 left, Arg1 right, Status* st) { static_assert(std::is_same::value && std::is_same::value, ""); T result = 0; if (ARROW_PREDICT_FALSE(MultiplyWithOverflow(left, right, &result))) { - ctx->SetStatus(Status::Invalid("overflow")); + *st = Status::Invalid("overflow"); } return result; } template - enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right) { + enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right, Status*) { static_assert(std::is_same::value && std::is_same::value, ""); return left * right; } @@ -191,16 +200,17 @@ struct MultiplyChecked { struct Divide { template - static enable_if_floating_point Call(KernelContext* ctx, Arg0 left, Arg1 right) { + static enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right, + Status*) { return left / right; } template - static enable_if_integer Call(KernelContext* ctx, Arg0 left, Arg1 right) { + static enable_if_integer Call(KernelContext*, Arg0 left, Arg1 right, Status* st) { T result; if (ARROW_PREDICT_FALSE(DivideWithOverflow(left, right, &result))) { if (right == 0) { - ctx->SetStatus(Status::Invalid("divide by zero")); + *st = Status::Invalid("divide by zero"); } else { result = 0; } @@ -211,24 +221,25 @@ struct Divide { struct DivideChecked { template - static enable_if_integer Call(KernelContext* ctx, Arg0 left, Arg1 right) { + static enable_if_integer Call(KernelContext*, Arg0 left, Arg1 right, Status* st) { static_assert(std::is_same::value && std::is_same::value, ""); T result; if (ARROW_PREDICT_FALSE(DivideWithOverflow(left, right, &result))) { if (right == 0) { - ctx->SetStatus(Status::Invalid("divide by zero")); + *st = Status::Invalid("divide by zero"); } else { - ctx->SetStatus(Status::Invalid("overflow")); + *st = Status::Invalid("overflow"); } } return result; } template - static enable_if_floating_point Call(KernelContext* ctx, Arg0 left, Arg1 right) { + static enable_if_floating_point Call(KernelContext*, Arg0 left, Arg1 right, + Status* st) { static_assert(std::is_same::value && std::is_same::value, ""); if (ARROW_PREDICT_FALSE(right == 0)) { - ctx->SetStatus(Status::Invalid("divide by zero")); + *st = Status::Invalid("divide by zero"); return 0; } return left / right; @@ -249,27 +260,25 @@ struct Power { } template - static enable_if_integer Call(KernelContext* ctx, T base, T exp) { + static enable_if_integer Call(KernelContext*, T base, T exp, Status* st) { if (exp < 0) { - ctx->SetStatus( - Status::Invalid("integers to negative integer powers are not allowed")); + *st = Status::Invalid("integers to negative integer powers are not allowed"); return 0; } return static_cast(IntegerPower(base, exp)); } template - static enable_if_floating_point Call(KernelContext* ctx, T base, T exp) { + static enable_if_floating_point Call(KernelContext*, T base, T exp, Status*) { return std::pow(base, exp); } }; struct PowerChecked { template - static enable_if_integer Call(KernelContext* ctx, Arg0 base, Arg1 exp) { + static enable_if_integer Call(KernelContext*, Arg0 base, Arg1 exp, Status* st) { if (exp < 0) { - ctx->SetStatus( - Status::Invalid("integers to negative integer powers are not allowed")); + *st = Status::Invalid("integers to negative integer powers are not allowed"); return 0; } else if (exp == 0) { return 1; @@ -287,13 +296,13 @@ struct PowerChecked { bitmask >>= 1; } if (overflow) { - ctx->SetStatus(Status::Invalid("overflow")); + *st = Status::Invalid("overflow"); } return pow; } template - static enable_if_floating_point Call(KernelContext* ctx, Arg0 base, Arg1 exp) { + static enable_if_floating_point Call(KernelContext*, Arg0 base, Arg1 exp, Status*) { static_assert(std::is_same::value && std::is_same::value, ""); return std::pow(base, exp); } diff --git a/cpp/src/arrow/compute/kernels/scalar_boolean.cc b/cpp/src/arrow/compute/kernels/scalar_boolean.cc index 009b968809c..d555a81392a 100644 --- a/cpp/src/arrow/compute/kernels/scalar_boolean.cc +++ b/cpp/src/arrow/compute/kernels/scalar_boolean.cc @@ -96,57 +96,62 @@ inline Bitmap GetBitmap(const ArrayData& arr, int index) { } struct Invert { - static void Call(KernelContext* ctx, const Scalar& in, Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) { *checked_cast(out) = InvertScalar(in); + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& in, ArrayData* out) { GetBitmap(*out, 1).CopyFromInverted(GetBitmap(in, 1)); + return Status::OK(); } }; template struct Commutative { - static void Call(KernelContext* ctx, const Scalar& left, const ArrayData& right, - ArrayData* out) { - Op::Call(ctx, right, left, out); + static Status Call(KernelContext* ctx, const Scalar& left, const ArrayData& right, + ArrayData* out) { + return Op::Call(ctx, right, left, out); } }; struct And : Commutative { using Commutative::Call; - static void Call(KernelContext* ctx, const Scalar& left, const Scalar& right, - Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& left, const Scalar& right, + Scalar* out) { if (left.is_valid && right.is_valid) { checked_cast(out)->value = checked_cast(left).value && checked_cast(right).value; } + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, - ArrayData* out) { - if (!right.is_valid) return; // all null case - - return checked_cast(right).value - ? GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)) - : GetBitmap(*out, 1).SetBitsTo(false); + static Status Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, + ArrayData* out) { + if (right.is_valid) { + checked_cast(right).value + ? GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)) + : GetBitmap(*out, 1).SetBitsTo(false); + } + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, + ArrayData* out) { ::arrow::internal::BitmapAnd(left.buffers[1]->data(), left.offset, right.buffers[1]->data(), right.offset, right.length, out->offset, out->buffers[1]->mutable_data()); + return Status::OK(); } }; struct KleeneAnd : Commutative { using Commutative::Call; - static void Call(KernelContext* ctx, const Scalar& left, const Scalar& right, - Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& left, const Scalar& right, + Scalar* out) { bool left_true = left.is_valid && checked_cast(left).value; bool left_false = left.is_valid && !checked_cast(left).value; @@ -155,21 +160,24 @@ struct KleeneAnd : Commutative { checked_cast(out)->value = left_true && right_true; out->is_valid = left_false || right_false || (left_true && right_true); + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, + ArrayData* out) { bool right_true = right.is_valid && checked_cast(right).value; bool right_false = right.is_valid && !checked_cast(right).value; if (right_false) { - return GetBitmap(*out, 0).SetBitsTo(true), - GetBitmap(*out, 1).SetBitsTo(false); // all false case + GetBitmap(*out, 0).SetBitsTo(true); + GetBitmap(*out, 1).SetBitsTo(false); // all false case + return Status::OK(); } if (right_true) { - return GetBitmap(*out, 0).CopyFrom(GetBitmap(left, 0)), - GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + GetBitmap(*out, 0).CopyFrom(GetBitmap(left, 0)); + GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + return Status::OK(); } // scalar was null: out[i] is valid iff left[i] was false @@ -178,10 +186,11 @@ struct KleeneAnd : Commutative { out->offset, out->buffers[0]->mutable_data()); ::arrow::internal::CopyBitmap(left.buffers[1]->data(), left.offset, left.length, out->buffers[1]->mutable_data(), out->offset); + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, + ArrayData* out) { if (left.GetNullCount() == 0 && right.GetNullCount() == 0) { GetBitmap(*out, 0).SetBitsTo(true); return And::Call(ctx, left, right, out); @@ -193,43 +202,47 @@ struct KleeneAnd : Commutative { *out_valid = left_false | right_false | (left_true & right_true); }; ComputeKleene(compute_word, ctx, left, right, out); + return Status::OK(); } }; struct Or : Commutative { using Commutative::Call; - static void Call(KernelContext* ctx, const Scalar& left, const Scalar& right, - Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& left, const Scalar& right, + Scalar* out) { if (left.is_valid && right.is_valid) { checked_cast(out)->value = checked_cast(left).value || checked_cast(right).value; } + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, - ArrayData* out) { - if (!right.is_valid) return; // all null case - - return checked_cast(right).value - ? GetBitmap(*out, 1).SetBitsTo(true) - : GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + static Status Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, + ArrayData* out) { + if (right.is_valid) { + checked_cast(right).value + ? GetBitmap(*out, 1).SetBitsTo(true) + : GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + } + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, + ArrayData* out) { ::arrow::internal::BitmapOr(left.buffers[1]->data(), left.offset, right.buffers[1]->data(), right.offset, right.length, out->offset, out->buffers[1]->mutable_data()); + return Status::OK(); } }; struct KleeneOr : Commutative { using Commutative::Call; - static void Call(KernelContext* ctx, const Scalar& left, const Scalar& right, - Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& left, const Scalar& right, + Scalar* out) { bool left_true = left.is_valid && checked_cast(left).value; bool left_false = left.is_valid && !checked_cast(left).value; @@ -238,21 +251,24 @@ struct KleeneOr : Commutative { checked_cast(out)->value = left_true || right_true; out->is_valid = left_true || right_true || (left_false && right_false); + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, + ArrayData* out) { bool right_true = right.is_valid && checked_cast(right).value; bool right_false = right.is_valid && !checked_cast(right).value; if (right_true) { - return GetBitmap(*out, 0).SetBitsTo(true), - GetBitmap(*out, 1).SetBitsTo(true); // all true case + GetBitmap(*out, 0).SetBitsTo(true); + GetBitmap(*out, 1).SetBitsTo(true); // all true case + return Status::OK(); } if (right_false) { - return GetBitmap(*out, 0).CopyFrom(GetBitmap(left, 0)), - GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + GetBitmap(*out, 0).CopyFrom(GetBitmap(left, 0)); + GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + return Status::OK(); } // scalar was null: out[i] is valid iff left[i] was true @@ -261,10 +277,11 @@ struct KleeneOr : Commutative { out->offset, out->buffers[0]->mutable_data()); ::arrow::internal::CopyBitmap(left.buffers[1]->data(), left.offset, left.length, out->buffers[1]->mutable_data(), out->offset); + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, + ArrayData* out) { if (left.GetNullCount() == 0 && right.GetNullCount() == 0) { GetBitmap(*out, 0).SetBitsTo(true); return Or::Call(ctx, left, right, out); @@ -277,86 +294,94 @@ struct KleeneOr : Commutative { *out_valid = left_true | right_true | (left_false & right_false); }; - return ComputeKleene(compute_word, ctx, left, right, out); + ComputeKleene(compute_word, ctx, left, right, out); + return Status::OK(); } }; struct Xor : Commutative { using Commutative::Call; - static void Call(KernelContext* ctx, const Scalar& left, const Scalar& right, - Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& left, const Scalar& right, + Scalar* out) { if (left.is_valid && right.is_valid) { checked_cast(out)->value = checked_cast(left).value ^ checked_cast(right).value; } + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, - ArrayData* out) { - if (!right.is_valid) return; // all null case - - return checked_cast(right).value - ? GetBitmap(*out, 1).CopyFromInverted(GetBitmap(left, 1)) - : GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + static Status Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, + ArrayData* out) { + if (right.is_valid) { + checked_cast(right).value + ? GetBitmap(*out, 1).CopyFromInverted(GetBitmap(left, 1)) + : GetBitmap(*out, 1).CopyFrom(GetBitmap(left, 1)); + } + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, + ArrayData* out) { ::arrow::internal::BitmapXor(left.buffers[1]->data(), left.offset, right.buffers[1]->data(), right.offset, right.length, out->offset, out->buffers[1]->mutable_data()); + return Status::OK(); } }; struct AndNot { - static void Call(KernelContext* ctx, const Scalar& left, const Scalar& right, - Scalar* out) { - And::Call(ctx, left, InvertScalar(right), out); + static Status Call(KernelContext* ctx, const Scalar& left, const Scalar& right, + Scalar* out) { + return And::Call(ctx, left, InvertScalar(right), out); } - static void Call(KernelContext* ctx, const Scalar& left, const ArrayData& right, - ArrayData* out) { - if (!left.is_valid) return; // all null case - - return checked_cast(left).value - ? GetBitmap(*out, 1).CopyFromInverted(GetBitmap(right, 1)) - : GetBitmap(*out, 1).SetBitsTo(false); + static Status Call(KernelContext* ctx, const Scalar& left, const ArrayData& right, + ArrayData* out) { + if (left.is_valid) { + checked_cast(left).value + ? GetBitmap(*out, 1).CopyFromInverted(GetBitmap(right, 1)) + : GetBitmap(*out, 1).SetBitsTo(false); + } + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, - ArrayData* out) { - And::Call(ctx, left, InvertScalar(right), out); + static Status Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, + ArrayData* out) { + return And::Call(ctx, left, InvertScalar(right), out); } - static void Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, + ArrayData* out) { ::arrow::internal::BitmapAndNot(left.buffers[1]->data(), left.offset, right.buffers[1]->data(), right.offset, right.length, out->offset, out->buffers[1]->mutable_data()); + return Status::OK(); } }; struct KleeneAndNot { - static void Call(KernelContext* ctx, const Scalar& left, const Scalar& right, - Scalar* out) { - KleeneAnd::Call(ctx, left, InvertScalar(right), out); + static Status Call(KernelContext* ctx, const Scalar& left, const Scalar& right, + Scalar* out) { + return KleeneAnd::Call(ctx, left, InvertScalar(right), out); } - static void Call(KernelContext* ctx, const Scalar& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const Scalar& left, const ArrayData& right, + ArrayData* out) { bool left_true = left.is_valid && checked_cast(left).value; bool left_false = left.is_valid && !checked_cast(left).value; if (left_false) { - return GetBitmap(*out, 0).SetBitsTo(true), - GetBitmap(*out, 1).SetBitsTo(false); // all false case + GetBitmap(*out, 0).SetBitsTo(true); + GetBitmap(*out, 1).SetBitsTo(false); // all false case + return Status::OK(); } if (left_true) { - return GetBitmap(*out, 0).CopyFrom(GetBitmap(right, 0)), - GetBitmap(*out, 1).CopyFromInverted(GetBitmap(right, 1)); + GetBitmap(*out, 0).CopyFrom(GetBitmap(right, 0)); + GetBitmap(*out, 1).CopyFromInverted(GetBitmap(right, 1)); + return Status::OK(); } // scalar was null: out[i] is valid iff right[i] was true @@ -365,15 +390,16 @@ struct KleeneAndNot { out->offset, out->buffers[0]->mutable_data()); ::arrow::internal::InvertBitmap(right.buffers[1]->data(), right.offset, right.length, out->buffers[1]->mutable_data(), out->offset); + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, - ArrayData* out) { - KleeneAnd::Call(ctx, left, InvertScalar(right), out); + static Status Call(KernelContext* ctx, const ArrayData& left, const Scalar& right, + ArrayData* out) { + return KleeneAnd::Call(ctx, left, InvertScalar(right), out); } - static void Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, - ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& left, const ArrayData& right, + ArrayData* out) { if (left.GetNullCount() == 0 && right.GetNullCount() == 0) { GetBitmap(*out, 0).SetBitsTo(true); return AndNot::Call(ctx, left, right, out); @@ -386,7 +412,8 @@ struct KleeneAndNot { *out_valid = left_false | right_true | (left_true & right_false); }; - return ComputeKleene(compute_word, ctx, left, right, out); + ComputeKleene(compute_word, ctx, left, right, out); + return Status::OK(); } }; diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc b/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc index e529d3791aa..dad94c1ace7 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc @@ -31,17 +31,17 @@ namespace internal { struct IsNonZero { template - static OutValue Call(KernelContext*, Arg0Value val) { + static OutValue Call(KernelContext*, Arg0Value val, Status*) { return val != 0; } }; struct ParseBooleanString { template - static OutValue Call(KernelContext* ctx, Arg0Value val) { + static OutValue Call(KernelContext*, Arg0Value val, Status* st) { bool result = false; if (ARROW_PREDICT_FALSE(!ParseValue(val.data(), val.size(), &result))) { - ctx->SetStatus(Status::Invalid("Failed to parse value: ", val)); + *st = Status::Invalid("Failed to parse value: ", val); } return result; } diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc b/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc index 7221722d53a..f42635c5dcd 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc @@ -148,7 +148,7 @@ void CastNumberToNumberUnsafe(Type::type in_type, Type::type out_type, const Dat // ---------------------------------------------------------------------- -void UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK(out->is_array()); DictionaryArray dict_arr(batch[0].array()); @@ -156,32 +156,32 @@ void UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& dict_type = *dict_arr.dictionary()->type(); if (!dict_type.Equals(options.to_type) && !CanCast(dict_type, *options.to_type)) { - ctx->SetStatus(Status::Invalid("Cast type ", options.to_type->ToString(), - " incompatible with dictionary type ", - dict_type.ToString())); - return; + return Status::Invalid("Cast type ", options.to_type->ToString(), + " incompatible with dictionary type ", dict_type.ToString()); } - KERNEL_ASSIGN_OR_RAISE(*out, ctx, - Take(Datum(dict_arr.dictionary()), Datum(dict_arr.indices()), - TakeOptions::Defaults(), ctx->exec_context())); + ARROW_ASSIGN_OR_RAISE(*out, + Take(Datum(dict_arr.dictionary()), Datum(dict_arr.indices()), + TakeOptions::Defaults(), ctx->exec_context())); if (!dict_type.Equals(options.to_type)) { - KERNEL_ASSIGN_OR_RAISE(*out, ctx, Cast(*out, options)); + ARROW_ASSIGN_OR_RAISE(*out, Cast(*out, options)); } + return Status::OK(); } -void OutputAllNull(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status OutputAllNull(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (out->is_scalar()) { out->scalar()->is_valid = false; - return; + } else { + ArrayData* output = out->mutable_array(); + output->buffers = {nullptr}; + output->null_count = batch.length; } - ArrayData* output = out->mutable_array(); - output->buffers = {nullptr}; - output->null_count = batch.length; + return Status::OK(); } -void CastFromExtension(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status CastFromExtension(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const CastOptions& options = checked_cast(ctx->state())->options; const DataType& in_type = *batch[0].type(); @@ -190,20 +190,20 @@ void CastFromExtension(KernelContext* ctx, const ExecBatch& batch, Datum* out) { ExtensionArray extension(batch[0].array()); Datum casted_storage; - KERNEL_RETURN_IF_ERROR( - ctx, Cast(*extension.storage(), out->type(), options, ctx->exec_context()) - .Value(&casted_storage)); + RETURN_NOT_OK(Cast(*extension.storage(), out->type(), options, ctx->exec_context()) + .Value(&casted_storage)); out->value = casted_storage.array(); + return Status::OK(); } -void CastFromNull(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - if (batch[0].is_scalar()) return; - - ArrayData* output = out->mutable_array(); - std::shared_ptr nulls; - Status s = MakeArrayOfNull(output->type, batch.length).Value(&nulls); - KERNEL_RETURN_IF_ERROR(ctx, s); - out->value = nulls->data(); +Status CastFromNull(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + if (!batch[0].is_scalar()) { + ArrayData* output = out->mutable_array(); + std::shared_ptr nulls; + RETURN_NOT_OK(MakeArrayOfNull(output->type, batch.length).Value(&nulls)); + out->value = nulls->data(); + } + return Status::OK(); } Result ResolveOutputFromOptions(KernelContext* ctx, @@ -223,7 +223,7 @@ Result ResolveOutputFromOptions(KernelContext* ctx, OutputType kOutputTargetType(ResolveOutputFromOptions); -void ZeroCopyCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ZeroCopyCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); // Make a copy of the buffers into a destination array without carrying // the type @@ -234,6 +234,7 @@ void ZeroCopyCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { output->buffers = input.buffers; output->offset = input.offset; output->child_data = input.child_data; + return Status::OK(); } void AddZeroCopyCast(Type::type in_type_id, InputType in_type, OutputType out_type, diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_internal.h b/cpp/src/arrow/compute/kernels/scalar_cast_internal.h index dabf0c2b061..2419d898a68 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_internal.h +++ b/cpp/src/arrow/compute/kernels/scalar_cast_internal.h @@ -37,10 +37,10 @@ struct CastFunctor {}; template struct CastFunctor< O, I, enable_if_t::value && is_parameter_free_type::value>> { - static void Exec(KernelContext*, const ExecBatch&, Datum*) {} + static Status Exec(KernelContext*, const ExecBatch&, Datum*) { return Status::OK(); } }; -void CastFromExtension(KernelContext* ctx, const ExecBatch& batch, Datum* out); +Status CastFromExtension(KernelContext* ctx, const ExecBatch& batch, Datum* out); // Utility for numeric casts void CastNumberToNumberUnsafe(Type::type in_type, Type::type out_type, const Datum& input, @@ -49,11 +49,11 @@ void CastNumberToNumberUnsafe(Type::type in_type, Type::type out_type, const Dat // ---------------------------------------------------------------------- // Dictionary to other things -void UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out); +Status UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out); -void OutputAllNull(KernelContext* ctx, const ExecBatch& batch, Datum* out); +Status OutputAllNull(KernelContext* ctx, const ExecBatch& batch, Datum* out); -void CastFromNull(KernelContext* ctx, const ExecBatch& batch, Datum* out); +Status CastFromNull(KernelContext* ctx, const ExecBatch& batch, Datum* out); // Adds a cast function where CastFunctor is specialized and the input and output // types are parameter free (have a type_singleton). Scalar inputs are handled by @@ -65,7 +65,7 @@ void AddSimpleCast(InputType in_ty, OutputType out_ty, CastFunction* func) { TrivialScalarUnaryAsArraysExec(CastFunctor::Exec))); } -void ZeroCopyCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* out); +Status ZeroCopyCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* out); void AddZeroCopyCast(Type::type in_type_id, InputType in_type, OutputType out_type, CastFunction* func); diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc b/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc index 1d81be48288..9364120c133 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc @@ -35,7 +35,7 @@ namespace compute { namespace internal { template -void CastListExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status CastListExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { using offset_type = typename Type::offset_type; using ScalarType = typename TypeTraits::ScalarType; @@ -49,13 +49,12 @@ void CastListExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK(!out_scalar->is_valid); if (in_scalar.is_valid) { - KERNEL_ASSIGN_OR_RAISE( - out_scalar->value, ctx, - Cast(*in_scalar.value, child_type, options, ctx->exec_context())); + ARROW_ASSIGN_OR_RAISE(out_scalar->value, Cast(*in_scalar.value, child_type, options, + ctx->exec_context())); out_scalar->is_valid = true; } - return; + return Status::OK(); } const ArrayData& in_array = *batch[0].array(); @@ -66,11 +65,11 @@ void CastListExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { Datum values = in_array.child_data[0]; if (in_array.offset != 0) { - KERNEL_ASSIGN_OR_RAISE(out_array->buffers[0], ctx, - CopyBitmap(ctx->memory_pool(), in_array.buffers[0]->data(), - in_array.offset, in_array.length)); - KERNEL_ASSIGN_OR_RAISE(out_array->buffers[1], ctx, - ctx->Allocate(sizeof(offset_type) * (in_array.length + 1))); + ARROW_ASSIGN_OR_RAISE(out_array->buffers[0], + CopyBitmap(ctx->memory_pool(), in_array.buffers[0]->data(), + in_array.offset, in_array.length)); + ARROW_ASSIGN_OR_RAISE(out_array->buffers[1], + ctx->Allocate(sizeof(offset_type) * (in_array.length + 1))); auto offsets = in_array.GetValues(1); auto shifted_offsets = out_array->GetMutableValues(1); @@ -81,11 +80,12 @@ void CastListExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { values = in_array.child_data[0]->Slice(offsets[0], offsets[in_array.length]); } - KERNEL_ASSIGN_OR_RAISE(Datum cast_values, ctx, - Cast(values, child_type, options, ctx->exec_context())); + ARROW_ASSIGN_OR_RAISE(Datum cast_values, + Cast(values, child_type, options, ctx->exec_context())); DCHECK_EQ(Datum::ARRAY, cast_values.kind()); out_array->child_data.push_back(cast_values.array()); + return Status::OK(); } template diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc b/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc index 160c4ce8857..cc7b533f262 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc @@ -36,16 +36,18 @@ using internal::ParseValue; namespace compute { namespace internal { -void CastIntegerToInteger(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status CastIntegerToInteger(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& options = checked_cast(ctx->state())->options; if (!options.allow_int_overflow) { - KERNEL_RETURN_IF_ERROR(ctx, IntegersCanFit(batch[0], *out->type())); + RETURN_NOT_OK(IntegersCanFit(batch[0], *out->type())); } CastNumberToNumberUnsafe(batch[0].type()->id(), out->type()->id(), batch[0], out); + return Status::OK(); } -void CastFloatingToFloating(KernelContext*, const ExecBatch& batch, Datum* out) { +Status CastFloatingToFloating(KernelContext*, const ExecBatch& batch, Datum* out) { CastNumberToNumberUnsafe(batch[0].type()->id(), out->type()->id(), batch[0], out); + return Status::OK(); } // ---------------------------------------------------------------------- @@ -168,12 +170,13 @@ Status CheckFloatToIntTruncation(const Datum& input, const Datum& output) { return Status::OK(); } -void CastFloatingToInteger(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status CastFloatingToInteger(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& options = checked_cast(ctx->state())->options; CastNumberToNumberUnsafe(batch[0].type()->id(), out->type()->id(), batch[0], out); if (!options.allow_float_truncate) { - KERNEL_RETURN_IF_ERROR(ctx, CheckFloatToIntTruncation(batch[0], *out)); + RETURN_NOT_OK(CheckFloatToIntTruncation(batch[0], *out)); } + return Status::OK(); } // ---------------------------------------------------------------------- @@ -246,13 +249,14 @@ Status CheckForIntegerToFloatingTruncation(const Datum& input, Type::type out_ty return Status::OK(); } -void CastIntegerToFloating(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status CastIntegerToFloating(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& options = checked_cast(ctx->state())->options; Type::type out_type = out->type()->id(); if (!options.allow_float_truncate) { - KERNEL_RETURN_IF_ERROR(ctx, CheckForIntegerToFloatingTruncation(batch[0], out_type)); + RETURN_NOT_OK(CheckForIntegerToFloatingTruncation(batch[0], out_type)); } CastNumberToNumberUnsafe(batch[0].type()->id(), out_type, batch[0], out); + return Status::OK(); } // ---------------------------------------------------------------------- @@ -260,7 +264,7 @@ void CastIntegerToFloating(KernelContext* ctx, const ExecBatch& batch, Datum* ou struct BooleanToNumber { template - static OutValue Call(KernelContext*, Arg0Value val) { + static OutValue Call(KernelContext*, Arg0Value val, Status*) { constexpr auto kOne = static_cast(1); constexpr auto kZero = static_cast(0); return val ? kOne : kZero; @@ -269,8 +273,9 @@ struct BooleanToNumber { template struct CastFunctor> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - applicator::ScalarUnary::Exec(ctx, batch, out); + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return applicator::ScalarUnary::Exec(ctx, batch, + out); } }; @@ -280,12 +285,11 @@ struct CastFunctor> { template struct ParseString { template - OutValue Call(KernelContext* ctx, Arg0Value val) const { + OutValue Call(KernelContext* ctx, Arg0Value val, Status* st) const { OutValue result = OutValue(0); if (ARROW_PREDICT_FALSE(!ParseValue(val.data(), val.size(), &result))) { - ctx->SetStatus(Status::Invalid("Failed to parse string: '", val, - "' as a scalar of type ", - TypeTraits::type_singleton()->ToString())); + *st = Status::Invalid("Failed to parse string: '", val, "' as a scalar of type ", + TypeTraits::type_singleton()->ToString()); } return result; } @@ -293,8 +297,8 @@ struct ParseString { template struct CastFunctor> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - applicator::ScalarUnaryNotNull>::Exec(ctx, batch, out); + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return applicator::ScalarUnaryNotNull>::Exec(ctx, batch, out); } }; @@ -303,12 +307,12 @@ struct CastFunctor> { struct DecimalToIntegerMixin { template - OutValue ToInteger(KernelContext* ctx, const Arg0Value& val) const { + OutValue ToInteger(KernelContext* ctx, const Arg0Value& val, Status* st) const { constexpr auto min_value = std::numeric_limits::min(); constexpr auto max_value = std::numeric_limits::max(); if (!allow_int_overflow_ && ARROW_PREDICT_FALSE(val < min_value || val > max_value)) { - ctx->SetStatus(Status::Invalid("Integer value out of bounds")); + *st = Status::Invalid("Integer value out of bounds"); return OutValue{}; // Zero } else { return static_cast(val.low_bits()); @@ -326,8 +330,8 @@ struct UnsafeUpscaleDecimalToInteger : public DecimalToIntegerMixin { using DecimalToIntegerMixin::DecimalToIntegerMixin; template - OutValue Call(KernelContext* ctx, Arg0Value val) const { - return ToInteger(ctx, val.IncreaseScaleBy(-in_scale_)); + OutValue Call(KernelContext* ctx, Arg0Value val, Status* st) const { + return ToInteger(ctx, val.IncreaseScaleBy(-in_scale_), st); } }; @@ -335,8 +339,8 @@ struct UnsafeDownscaleDecimalToInteger : public DecimalToIntegerMixin { using DecimalToIntegerMixin::DecimalToIntegerMixin; template - OutValue Call(KernelContext* ctx, Arg0Value val) const { - return ToInteger(ctx, val.ReduceScaleBy(in_scale_, false)); + OutValue Call(KernelContext* ctx, Arg0Value val, Status* st) const { + return ToInteger(ctx, val.ReduceScaleBy(in_scale_, false), st); } }; @@ -344,13 +348,13 @@ struct SafeRescaleDecimalToInteger : public DecimalToIntegerMixin { using DecimalToIntegerMixin::DecimalToIntegerMixin; template - OutValue Call(KernelContext* ctx, Arg0Value val) const { + OutValue Call(KernelContext* ctx, Arg0Value val, Status* st) const { auto result = val.Rescale(in_scale_, 0); if (ARROW_PREDICT_FALSE(!result.ok())) { - ctx->SetStatus(result.status()); + *st = result.status(); return OutValue{}; // Zero } else { - return ToInteger(ctx, *result); + return ToInteger(ctx, *result, st); } } }; @@ -360,7 +364,7 @@ struct CastFunctor::value && is_decimal_type::value>> { using out_type = typename O::c_type; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& options = checked_cast(ctx->state())->options; const auto& in_type_inst = checked_cast(*batch[0].type()); @@ -420,7 +424,7 @@ struct DecimalConversions { struct UnsafeUpscaleDecimal { template - OutValue Call(KernelContext* ctx, Arg0Value val) const { + OutValue Call(KernelContext*, Arg0Value val, Status*) const { using Conv = DecimalConversions; return Conv::ConvertOutput(Conv::ConvertInput(std::move(val)).IncreaseScaleBy(by_)); } @@ -429,7 +433,7 @@ struct UnsafeUpscaleDecimal { struct UnsafeDownscaleDecimal { template - OutValue Call(KernelContext* ctx, Arg0Value val) const { + OutValue Call(KernelContext*, Arg0Value val, Status*) const { using Conv = DecimalConversions; return Conv::ConvertOutput( Conv::ConvertInput(std::move(val)).ReduceScaleBy(by_, false)); @@ -439,12 +443,12 @@ struct UnsafeDownscaleDecimal { struct SafeRescaleDecimal { template - OutValue Call(KernelContext* ctx, Arg0Value val) const { + OutValue Call(KernelContext*, Arg0Value val, Status* st) const { using Conv = DecimalConversions; auto maybe_rescaled = Conv::ConvertInput(std::move(val)).Rescale(in_scale_, out_scale_); if (ARROW_PREDICT_FALSE(!maybe_rescaled.ok())) { - ctx->SetStatus(maybe_rescaled.status()); + *st = maybe_rescaled.status(); return {}; // Zero } @@ -452,8 +456,7 @@ struct SafeRescaleDecimal { return Conv::ConvertOutput(maybe_rescaled.MoveValueUnsafe()); } - ctx->SetStatus( - Status::Invalid("Decimal value does not fit in precision ", out_precision_)); + *st = Status::Invalid("Decimal value does not fit in precision ", out_precision_); return {}; // Zero } @@ -463,7 +466,7 @@ struct SafeRescaleDecimal { template struct CastFunctor::value && is_decimal_type::value>> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& options = checked_cast(ctx->state())->options; const auto& in_type = checked_cast(*batch[0].type()); @@ -497,7 +500,7 @@ struct CastFunctor - OutValue Call(KernelContext* ctx, RealType val) const { + OutValue Call(KernelContext*, RealType val, Status* st) const { auto maybe_decimal = OutValue::FromReal(val, out_precision_, out_scale_); if (ARROW_PREDICT_TRUE(maybe_decimal.ok())) { @@ -505,7 +508,7 @@ struct RealToDecimal { } if (!allow_truncate_) { - ctx->SetStatus(maybe_decimal.status()); + *st = maybe_decimal.status(); } return {}; // Zero } @@ -517,7 +520,7 @@ struct RealToDecimal { template struct CastFunctor::value && is_floating_type::value>> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& options = checked_cast(ctx->state())->options; const auto& out_type = checked_cast(*out->type()); const auto out_scale = out_type.scale(); @@ -534,7 +537,7 @@ struct CastFunctor - RealType Call(KernelContext* ctx, const Arg0Value& val) const { + RealType Call(KernelContext*, const Arg0Value& val, Status*) const { return val.template ToReal(in_scale_); } @@ -544,7 +547,7 @@ struct DecimalToReal { template struct CastFunctor::value && is_decimal_type::value>> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& in_type = checked_cast(*batch[0].type()); const auto in_scale = in_type.scale(); diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_string.cc b/cpp/src/arrow/compute/kernels/scalar_cast_string.cc index 6f965a46676..3ce537b7223 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_string.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_string.cc @@ -48,11 +48,11 @@ struct NumericToStringCastFunctor { using BuilderType = typename TypeTraits::BuilderType; using FormatterType = StringFormatter; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK(out->is_array()); const ArrayData& input = *batch[0].array(); ArrayData* output = out->mutable_array(); - ctx->SetStatus(Convert(ctx, input, output)); + return Convert(ctx, input, output); } static Status Convert(KernelContext* ctx, const ArrayData& input, ArrayData* output) { @@ -94,33 +94,35 @@ struct Utf8Validator { }; template -void CastBinaryToBinaryOffsets(KernelContext* ctx, const ArrayData& input, - ArrayData* output) { +Status CastBinaryToBinaryOffsets(KernelContext* ctx, const ArrayData& input, + ArrayData* output) { static_assert(std::is_same::value, "Cast same-width offsets (no-op)"); + return Status::OK(); } // Upcast offsets template <> -void CastBinaryToBinaryOffsets(KernelContext* ctx, - const ArrayData& input, - ArrayData* output) { +Status CastBinaryToBinaryOffsets(KernelContext* ctx, + const ArrayData& input, + ArrayData* output) { using input_offset_type = int32_t; using output_offset_type = int64_t; - KERNEL_ASSIGN_OR_RAISE( - output->buffers[1], ctx, + ARROW_ASSIGN_OR_RAISE( + output->buffers[1], ctx->Allocate((output->length + output->offset + 1) * sizeof(output_offset_type))); memset(output->buffers[1]->mutable_data(), 0, output->offset * sizeof(output_offset_type)); ::arrow::internal::CastInts(input.GetValues(1), output->GetMutableValues(1), output->length + 1); + return Status::OK(); } // Downcast offsets template <> -void CastBinaryToBinaryOffsets(KernelContext* ctx, - const ArrayData& input, - ArrayData* output) { +Status CastBinaryToBinaryOffsets(KernelContext* ctx, + const ArrayData& input, + ArrayData* output) { using input_offset_type = int64_t; using output_offset_type = int32_t; @@ -130,22 +132,23 @@ void CastBinaryToBinaryOffsets(KernelContext* ctx, // Binary offsets are ascending, so it's enough to check the last one for overflow. if (input_offsets[input.length] > kMaxOffset) { - ctx->SetStatus(Status::Invalid("Failed casting from ", input.type->ToString(), " to ", - output->type->ToString(), ": input array too large")); + return Status::Invalid("Failed casting from ", input.type->ToString(), " to ", + output->type->ToString(), ": input array too large"); } else { - KERNEL_ASSIGN_OR_RAISE(output->buffers[1], ctx, - ctx->Allocate((output->length + output->offset + 1) * - sizeof(output_offset_type))); + ARROW_ASSIGN_OR_RAISE(output->buffers[1], + ctx->Allocate((output->length + output->offset + 1) * + sizeof(output_offset_type))); memset(output->buffers[1]->mutable_data(), 0, output->offset * sizeof(output_offset_type)); ::arrow::internal::CastInts(input.GetValues(1), output->GetMutableValues(1), output->length + 1); + return Status::OK(); } } template -void BinaryToBinaryCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status BinaryToBinaryCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK(out->is_array()); const CastOptions& options = checked_cast(*ctx->state()).options; const ArrayData& input = *batch[0].array(); @@ -155,17 +158,12 @@ void BinaryToBinaryCastExec(KernelContext* ctx, const ExecBatch& batch, Datum* o ArrayDataVisitor visitor; Utf8Validator validator; - Status st = visitor.Visit(input, &validator); - if (!st.ok()) { - ctx->SetStatus(st); - return; - } + RETURN_NOT_OK(visitor.Visit(input, &validator)); } // Start with a zero-copy cast, but change indices to expected size - ZeroCopyCastExec(ctx, batch, out); - - CastBinaryToBinaryOffsets( + RETURN_NOT_OK(ZeroCopyCastExec(ctx, batch, out)); + return CastBinaryToBinaryOffsets( ctx, input, out->mutable_array()); } diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc b/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc index d7d1faf7ae5..1a58fce7c74 100644 --- a/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc +++ b/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc @@ -39,8 +39,8 @@ constexpr int64_t kMillisecondsInDay = 86400000; // From one timestamp to another template -void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, - const int64_t factor, const ArrayData& input, ArrayData* output) { +Status ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, + const int64_t factor, const ArrayData& input, ArrayData* output) { const CastOptions& options = checked_cast(*ctx->state()).options; auto in_data = input.GetValues(1); auto out_data = output->GetMutableValues(1); @@ -55,10 +55,10 @@ void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, out_data[i] = static_cast(in_data[i] * factor); } } else { -#define RAISE_OVERFLOW_CAST(VAL) \ - ctx->SetStatus(Status::Invalid("Casting from ", input.type->ToString(), " to ", \ - output->type->ToString(), " would result in ", \ - "out of bounds timestamp: ", VAL)); +#define RAISE_OVERFLOW_CAST(VAL) \ + return Status::Invalid("Casting from ", input.type->ToString(), " to ", \ + output->type->ToString(), " would result in ", \ + "out of bounds timestamp: ", VAL); int64_t max_val = std::numeric_limits::max() / factor; int64_t min_val = std::numeric_limits::min() / factor; @@ -67,7 +67,6 @@ void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, for (int64_t i = 0; i < input.length; i++) { if (bit_reader.IsSet() && (in_data[i] < min_val || in_data[i] > max_val)) { RAISE_OVERFLOW_CAST(in_data[i]); - break; } out_data[i] = static_cast(in_data[i] * factor); bit_reader.Next(); @@ -76,7 +75,6 @@ void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, for (int64_t i = 0; i < input.length; i++) { if (in_data[i] < min_val || in_data[i] > max_val) { RAISE_OVERFLOW_CAST(in_data[i]); - break; } out_data[i] = static_cast(in_data[i] * factor); } @@ -90,9 +88,9 @@ void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, out_data[i] = static_cast(in_data[i] / factor); } } else { -#define RAISE_INVALID_CAST(VAL) \ - ctx->SetStatus(Status::Invalid("Casting from ", input.type->ToString(), " to ", \ - output->type->ToString(), " would lose data: ", VAL)); +#define RAISE_INVALID_CAST(VAL) \ + return Status::Invalid("Casting from ", input.type->ToString(), " to ", \ + output->type->ToString(), " would lose data: ", VAL); if (input.null_count != 0) { BitmapReader bit_reader(input.buffers[0]->data(), input.offset, input.length); @@ -100,7 +98,6 @@ void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, out_data[i] = static_cast(in_data[i] / factor); if (bit_reader.IsSet() && (out_data[i] * factor != in_data[i])) { RAISE_INVALID_CAST(in_data[i]); - break; } bit_reader.Next(); } @@ -109,7 +106,6 @@ void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, out_data[i] = static_cast(in_data[i] / factor); if (out_data[i] * factor != in_data[i]) { RAISE_INVALID_CAST(in_data[i]); - break; } } } @@ -117,6 +113,8 @@ void ShiftTime(KernelContext* ctx, const util::DivideOrMultiply factor_op, #undef RAISE_INVALID_CAST } } + + return Status::OK(); } // and @@ -125,7 +123,7 @@ struct CastFunctor< O, I, enable_if_t<(is_timestamp_type::value && is_timestamp_type::value) || (is_duration_type::value && is_duration_type::value)>> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); const ArrayData& input = *batch[0].array(); @@ -139,13 +137,14 @@ struct CastFunctor< // lengths to make this zero copy in the future but we leave it for now auto conversion = util::GetTimestampConversion(in_type.unit(), out_type.unit()); - ShiftTime(ctx, conversion.first, conversion.second, input, output); + return ShiftTime(ctx, conversion.first, conversion.second, input, + output); } }; template <> struct CastFunctor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); const ArrayData& input = *batch[0].array(); @@ -161,13 +160,13 @@ struct CastFunctor { }; const int64_t factor = kTimestampToDateFactors[static_cast(in_type.unit())]; - ShiftTime(ctx, util::DIVIDE, factor, input, output); + return ShiftTime(ctx, util::DIVIDE, factor, input, output); } }; template <> struct CastFunctor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); const CastOptions& options = checked_cast(*ctx->state()).options; @@ -176,10 +175,8 @@ struct CastFunctor { const auto& in_type = checked_cast(*input.type); auto conversion = util::GetTimestampConversion(in_type.unit(), TimeUnit::MILLI); - ShiftTime(ctx, conversion.first, conversion.second, input, output); - if (!ctx->status().ok()) { - return; - } + RETURN_NOT_OK((ShiftTime(ctx, conversion.first, conversion.second, + input, output))); // Ensure that intraday milliseconds have been zeroed out auto out_data = output->GetMutableValues(1); @@ -191,9 +188,7 @@ struct CastFunctor { const int64_t remainder = out_data[i] % kMillisecondsInDay; if (ARROW_PREDICT_FALSE(!options.allow_time_truncate && bit_reader.IsSet() && remainder > 0)) { - ctx->SetStatus( - Status::Invalid("Timestamp value had non-zero intraday milliseconds")); - break; + return Status::Invalid("Timestamp value had non-zero intraday milliseconds"); } out_data[i] -= remainder; bit_reader.Next(); @@ -202,13 +197,13 @@ struct CastFunctor { for (int64_t i = 0; i < input.length; ++i) { const int64_t remainder = out_data[i] % kMillisecondsInDay; if (ARROW_PREDICT_FALSE(!options.allow_time_truncate && remainder > 0)) { - ctx->SetStatus( - Status::Invalid("Timestamp value had non-zero intraday milliseconds")); - break; + return Status::Invalid("Timestamp value had non-zero intraday milliseconds"); } out_data[i] -= remainder; } } + + return Status::OK(); } }; @@ -220,7 +215,7 @@ struct CastFunctor::value && is_time_type:: using in_t = typename I::c_type; using out_t = typename O::c_type; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); const ArrayData& input = *batch[0].array(); @@ -231,7 +226,8 @@ struct CastFunctor::value && is_time_type:: const auto& out_type = checked_cast(*output->type); DCHECK_NE(in_type.unit(), out_type.unit()) << "Do not cast equal types"; auto conversion = util::GetTimestampConversion(in_type.unit(), out_type.unit()); - ShiftTime(ctx, conversion.first, conversion.second, input, output); + return ShiftTime(ctx, conversion.first, conversion.second, input, + output); } }; @@ -240,21 +236,21 @@ struct CastFunctor::value && is_time_type:: template <> struct CastFunctor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); - ShiftTime(ctx, util::MULTIPLY, kMillisecondsInDay, - *batch[0].array(), out->mutable_array()); + return ShiftTime(ctx, util::MULTIPLY, kMillisecondsInDay, + *batch[0].array(), out->mutable_array()); } }; template <> struct CastFunctor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); - ShiftTime(ctx, util::DIVIDE, kMillisecondsInDay, *batch[0].array(), - out->mutable_array()); + return ShiftTime(ctx, util::DIVIDE, kMillisecondsInDay, + *batch[0].array(), out->mutable_array()); } }; @@ -263,7 +259,7 @@ struct CastFunctor { template <> struct CastFunctor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); const auto& out_type = checked_cast(*out->type()); @@ -273,22 +269,22 @@ struct CastFunctor { // multiply to achieve days -> unit conversion.second *= kMillisecondsInDay / 1000; - ShiftTime(ctx, util::MULTIPLY, conversion.second, *batch[0].array(), - out->mutable_array()); + return ShiftTime(ctx, util::MULTIPLY, conversion.second, + *batch[0].array(), out->mutable_array()); } }; template <> struct CastFunctor { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DCHECK_EQ(batch[0].kind(), Datum::ARRAY); const auto& out_type = checked_cast(*out->type()); // date64 is ms since epoch auto conversion = util::GetTimestampConversion(TimeUnit::MILLI, out_type.unit()); - ShiftTime(ctx, conversion.first, conversion.second, - *batch[0].array(), out->mutable_array()); + return ShiftTime(ctx, conversion.first, conversion.second, + *batch[0].array(), out->mutable_array()); } }; @@ -297,11 +293,11 @@ struct CastFunctor { struct ParseTimestamp { template - OutValue Call(KernelContext* ctx, Arg0Value val) const { + OutValue Call(KernelContext*, Arg0Value val, Status* st) const { OutValue result = 0; if (ARROW_PREDICT_FALSE(!ParseValue(type, val.data(), val.size(), &result))) { - ctx->SetStatus(Status::Invalid("Failed to parse string: '", val, - "' as a scalar of type ", type.ToString())); + *st = Status::Invalid("Failed to parse string: '", val, "' as a scalar of type ", + type.ToString()); } return result; } @@ -311,7 +307,7 @@ struct ParseTimestamp { template struct CastFunctor::value>> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& out_type = checked_cast(*out->type()); applicator::ScalarUnaryNotNullStateful kernel( ParseTimestamp{out_type}); diff --git a/cpp/src/arrow/compute/kernels/scalar_compare.cc b/cpp/src/arrow/compute/kernels/scalar_compare.cc index 58d3e6fc781..8da97ef2260 100644 --- a/cpp/src/arrow/compute/kernels/scalar_compare.cc +++ b/cpp/src/arrow/compute/kernels/scalar_compare.cc @@ -30,28 +30,28 @@ namespace { struct Equal { template - static constexpr bool Call(KernelContext*, const T& left, const T& right) { + static constexpr bool Call(KernelContext*, const T& left, const T& right, Status*) { return left == right; } }; struct NotEqual { template - static constexpr bool Call(KernelContext*, const T& left, const T& right) { + static constexpr bool Call(KernelContext*, const T& left, const T& right, Status*) { return left != right; } }; struct Greater { template - static constexpr bool Call(KernelContext*, const T& left, const T& right) { + static constexpr bool Call(KernelContext*, const T& left, const T& right, Status*) { return left > right; } }; struct GreaterEqual { template - static constexpr bool Call(KernelContext*, const T& left, const T& right) { + static constexpr bool Call(KernelContext*, const T& left, const T& right, Status*) { return left >= right; } }; diff --git a/cpp/src/arrow/compute/kernels/scalar_fill_null.cc b/cpp/src/arrow/compute/kernels/scalar_fill_null.cc index 9624f88e68f..f52e0045470 100644 --- a/cpp/src/arrow/compute/kernels/scalar_fill_null.cc +++ b/cpp/src/arrow/compute/kernels/scalar_fill_null.cc @@ -43,7 +43,7 @@ template struct FillNullFunctor::value>> { using T = typename TypeTraits::CType; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const ArrayData& data = *batch[0].array(); const Scalar& fill_value = *batch[1].scalar(); ArrayData* output = out->mutable_array(); @@ -54,8 +54,8 @@ struct FillNullFunctor::value>> { T value = UnboxScalar::Unbox(fill_value); if (data.MayHaveNulls() != 0 && fill_value.is_valid) { - KERNEL_ASSIGN_OR_RAISE(std::shared_ptr out_buf, ctx, - ctx->Allocate(data.length * sizeof(T))); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr out_buf, + ctx->Allocate(data.length * sizeof(T))); const uint8_t* is_valid = data.buffers[0]->data(); const T* in_values = data.GetValues(1); @@ -83,6 +83,8 @@ struct FillNullFunctor::value>> { } else { *output = data; } + + return Status::OK(); } }; @@ -90,15 +92,15 @@ struct FillNullFunctor::value>> { template struct FillNullFunctor::value>> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const ArrayData& data = *batch[0].array(); const Scalar& fill_value = *batch[1].scalar(); ArrayData* output = out->mutable_array(); bool value = UnboxScalar::Unbox(fill_value); if (data.MayHaveNulls() != 0 && fill_value.is_valid) { - KERNEL_ASSIGN_OR_RAISE(std::shared_ptr out_buf, ctx, - ctx->AllocateBitmap(data.length)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr out_buf, + ctx->AllocateBitmap(data.length)); const uint8_t* is_valid = data.buffers[0]->data(); const uint8_t* data_bitmap = data.buffers[1]->data(); @@ -132,6 +134,8 @@ struct FillNullFunctor::value>> { } else { *output = data; } + + return Status::OK(); } }; @@ -139,9 +143,10 @@ struct FillNullFunctor::value>> { template struct FillNullFunctor::value>> { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // Nothing preallocated, so we assign into the output *out->mutable_array() = *batch[0].array(); + return Status::OK(); } }; @@ -151,7 +156,7 @@ template struct FillNullFunctor::value>> { using BuilderType = typename TypeTraits::BuilderType; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const ArrayData& input = *batch[0].array(); const auto& fill_value_scalar = checked_cast(*batch[1].scalar()); @@ -166,22 +171,15 @@ struct FillNullFunctor::value>> { if (null_count > 0 && fill_value_scalar.is_valid) { BuilderType builder(input.type, ctx->memory_pool()); - KERNEL_RETURN_IF_ERROR(ctx, builder.ReserveData(input.buffers[2]->size() + - fill_value.length() * null_count)); - KERNEL_RETURN_IF_ERROR(ctx, builder.Resize(input.length)); - - KERNEL_RETURN_IF_ERROR(ctx, VisitArrayDataInline( - input, - [&](util::string_view s) { - builder.UnsafeAppend(s); - return Status::OK(); - }, - [&]() { - builder.UnsafeAppend(fill_value); - return Status::OK(); - })); + RETURN_NOT_OK(builder.ReserveData(input.buffers[2]->size() + + fill_value.length() * null_count)); + RETURN_NOT_OK(builder.Resize(input.length)); + + VisitArrayDataInline( + input, [&](util::string_view s) { builder.UnsafeAppend(s); }, + [&]() { builder.UnsafeAppend(fill_value); }); std::shared_ptr string_array; - KERNEL_RETURN_IF_ERROR(ctx, builder.Finish(&string_array)); + RETURN_NOT_OK(builder.Finish(&string_array)); *output = *string_array->data(); // The builder does not match the logical type, due to // GenerateTypeAgnosticVarBinaryBase @@ -189,6 +187,8 @@ struct FillNullFunctor::value>> { } else { *output = input; } + + return Status::OK(); } }; diff --git a/cpp/src/arrow/compute/kernels/scalar_nested.cc b/cpp/src/arrow/compute/kernels/scalar_nested.cc index 8a6a69932c0..e4ab3f9b418 100644 --- a/cpp/src/arrow/compute/kernels/scalar_nested.cc +++ b/cpp/src/arrow/compute/kernels/scalar_nested.cc @@ -29,7 +29,7 @@ namespace internal { namespace { template -void ListValueLength(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ListValueLength(KernelContext* ctx, const ExecBatch& batch, Datum* out) { using ScalarType = typename TypeTraits::ScalarType; using OffsetScalarType = typename TypeTraits::OffsetScalarType; @@ -51,6 +51,8 @@ void ListValueLength(KernelContext* ctx, const ExecBatch& batch, Datum* out) { static_cast(arg0.value->length()); } } + + return Status::OK(); } const FunctionDoc list_value_length_doc{ @@ -99,16 +101,15 @@ Result ProjectResolve(KernelContext* ctx, return ValueDescr{struct_(std::move(fields)), shape}; } -void ProjectExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - KERNEL_ASSIGN_OR_RAISE(auto descr, ctx, ProjectResolve(ctx, batch.GetDescriptors())); +Status ProjectExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + ARROW_ASSIGN_OR_RAISE(auto descr, ProjectResolve(ctx, batch.GetDescriptors())); for (int i = 0; i < batch.num_values(); ++i) { const auto& field = checked_cast(*descr.type).field(i); if (batch[i].null_count() > 0 && !field->nullable()) { - ctx->SetStatus(Status::Invalid("Output field ", field, " (#", i, - ") does not allow nulls but the corresponding " - "argument was not entirely valid.")); - return; + return Status::Invalid("Output field ", field, " (#", i, + ") does not allow nulls but the corresponding " + "argument was not entirely valid."); } } @@ -120,7 +121,7 @@ void ProjectExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { *out = Datum(std::make_shared(std::move(scalars), std::move(descr.type))); - return; + return Status::OK(); } ArrayVector arrays(batch.num_values()); @@ -130,12 +131,12 @@ void ProjectExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { continue; } - KERNEL_ASSIGN_OR_RAISE( - arrays[i], ctx, - MakeArrayFromScalar(*batch[i].scalar(), batch.length, ctx->memory_pool())); + ARROW_ASSIGN_OR_RAISE(arrays[i], MakeArrayFromScalar(*batch[i].scalar(), batch.length, + ctx->memory_pool())); } *out = std::make_shared(descr.type, batch.length, std::move(arrays)); + return Status::OK(); } const FunctionDoc project_doc{"Wrap Arrays into a StructArray", diff --git a/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc b/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc index ffc1e11a7be..2868b0c743f 100644 --- a/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc +++ b/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc @@ -154,7 +154,7 @@ struct InitStateVisitor { // Handle Decimal128Type, FixedSizeBinaryType Status Visit(const FixedSizeBinaryType& type) { return Init(); } - Status GetResult(std::unique_ptr* out) { + Result> GetResult() { if (!options.value_set.type()->Equals(arg_type)) { ARROW_ASSIGN_OR_RAISE( options.value_set, @@ -162,22 +162,18 @@ struct InitStateVisitor { } RETURN_NOT_OK(VisitTypeInline(*arg_type, this)); - *out = std::move(result); - return Status::OK(); + return std::move(result); } }; -std::unique_ptr InitSetLookup(KernelContext* ctx, - const KernelInitArgs& args) { +Result> InitSetLookup(KernelContext* ctx, + const KernelInitArgs& args) { if (args.options == nullptr) { - ctx->SetStatus(Status::Invalid( - "Attempted to call a set lookup function without SetLookupOptions")); - return nullptr; + return Status::Invalid( + "Attempted to call a set lookup function without SetLookupOptions"); } - std::unique_ptr result; - ctx->SetStatus(InitStateVisitor{ctx, args}.GetResult(&result)); - return result; + return InitStateVisitor{ctx, args}.GetResult(); } struct IndexInVisitor { @@ -271,8 +267,8 @@ struct IndexInVisitor { } }; -void ExecIndexIn(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - KERNEL_RETURN_IF_ERROR(ctx, IndexInVisitor(ctx, *batch[0].array(), out).Execute()); +Status ExecIndexIn(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return IndexInVisitor(ctx, *batch[0].array(), out).Execute(); } // ---------------------------------------------------------------------- @@ -351,8 +347,8 @@ struct IsInVisitor { Status Execute() { return VisitTypeInline(*data.type, this); } }; -void ExecIsIn(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - KERNEL_RETURN_IF_ERROR(ctx, IsInVisitor(ctx, *batch[0].array(), out).Execute()); +Status ExecIsIn(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return IsInVisitor(ctx, *batch[0].array(), out).Execute(); } // Unary set lookup kernels available for the following input types diff --git a/cpp/src/arrow/compute/kernels/scalar_string.cc b/cpp/src/arrow/compute/kernels/scalar_string.cc index d5473749fe1..065c512c552 100644 --- a/cpp/src/arrow/compute/kernels/scalar_string.cc +++ b/cpp/src/arrow/compute/kernels/scalar_string.cc @@ -87,14 +87,14 @@ static inline bool IsAsciiCharacter(T character) { struct BinaryLength { template - static OutValue Call(KernelContext*, Arg0Value val) { + static OutValue Call(KernelContext*, Arg0Value val, Status*) { return static_cast(val.size()); } }; struct Utf8Length { template - static OutValue Call(KernelContext*, Arg0Value val) { + static OutValue Call(KernelContext*, Arg0Value val, Status*) { auto str = reinterpret_cast(val.data()); auto strlen = val.size(); @@ -139,10 +139,10 @@ struct StringTransform { using ArrayType = typename TypeTraits::ArrayType; static int64_t MaxCodeunits(offset_type input_ncodeunits) { return input_ncodeunits; } - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - Derived().Execute(ctx, batch, out); + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return Derived().Execute(ctx, batch, out); } - void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (batch[0].kind() == Datum::ARRAY) { const ArrayData& input = *batch[0].array(); ArrayType input_boxed(batch[0].array()); @@ -153,13 +153,11 @@ struct StringTransform { int64_t output_ncodeunits_max = Derived::MaxCodeunits(input_ncodeunits); if (output_ncodeunits_max > std::numeric_limits::max()) { - ctx->SetStatus(Status::CapacityError( - "Result might not fit in a 32bit utf8 array, convert to large_utf8")); - return; + return Status::CapacityError( + "Result might not fit in a 32bit utf8 array, convert to large_utf8"); } - KERNEL_ASSIGN_OR_RAISE(auto values_buffer, ctx, - ctx->Allocate(output_ncodeunits_max)); + ARROW_ASSIGN_OR_RAISE(auto values_buffer, ctx->Allocate(output_ncodeunits_max)); output->buffers[2] = values_buffer; // String offsets are preallocated @@ -175,16 +173,14 @@ struct StringTransform { if (ARROW_PREDICT_FALSE(!static_cast(*this).Transform( input_string, input_string_ncodeunits, output_str + output_ncodeunits, &encoded_nbytes))) { - ctx->SetStatus(Status::Invalid("Invalid UTF8 sequence in input")); - return; + return Status::Invalid("Invalid UTF8 sequence in input"); } output_ncodeunits += encoded_nbytes; output_string_offsets[i + 1] = output_ncodeunits; } // Trim the codepoint buffer, since we allocated too much - KERNEL_RETURN_IF_ERROR( - ctx, values_buffer->Resize(output_ncodeunits, /*shrink_to_fit=*/true)); + RETURN_NOT_OK(values_buffer->Resize(output_ncodeunits, /*shrink_to_fit=*/true)); } else { const auto& input = checked_cast(*batch[0].scalar()); auto result = checked_pointer_cast(MakeNullScalar(out->type())); @@ -194,25 +190,23 @@ struct StringTransform { int64_t output_ncodeunits_max = Derived::MaxCodeunits(data_nbytes); if (output_ncodeunits_max > std::numeric_limits::max()) { - ctx->SetStatus(Status::CapacityError( - "Result might not fit in a 32bit utf8 array, convert to large_utf8")); - return; + return Status::CapacityError( + "Result might not fit in a 32bit utf8 array, convert to large_utf8"); } - KERNEL_ASSIGN_OR_RAISE(auto value_buffer, ctx, - ctx->Allocate(output_ncodeunits_max)); + ARROW_ASSIGN_OR_RAISE(auto value_buffer, ctx->Allocate(output_ncodeunits_max)); result->value = value_buffer; offset_type encoded_nbytes = 0; if (ARROW_PREDICT_FALSE(!static_cast(*this).Transform( input.value->data(), data_nbytes, value_buffer->mutable_data(), &encoded_nbytes))) { - ctx->SetStatus(Status::Invalid("Invalid UTF8 sequence in input")); - return; + return Status::Invalid("Invalid UTF8 sequence in input"); } - KERNEL_RETURN_IF_ERROR( - ctx, value_buffer->Resize(encoded_nbytes, /*shrink_to_fit=*/true)); + RETURN_NOT_OK(value_buffer->Resize(encoded_nbytes, /*shrink_to_fit=*/true)); } out->value = result; } + + return Status::OK(); } }; @@ -244,9 +238,9 @@ struct StringTransformCodepoint : StringTransform { // two code units (even) can grow to 3 code units. return static_cast(input_ncodeunits) * 3 / 2; } - void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { EnsureLookupTablesFilled(); - Base::Execute(ctx, batch, out); + return Base::Execute(ctx, batch, out); } }; @@ -293,8 +287,8 @@ Status GetShiftedOffsets(KernelContext* ctx, const Buffer& input_buffer, int64_t // Apply `transform` to input character data- this function cannot change the // length template -void StringDataTransform(KernelContext* ctx, const ExecBatch& batch, - TransformFunc transform, Datum* out) { +Status StringDataTransform(KernelContext* ctx, const ExecBatch& batch, + TransformFunc transform, Datum* out) { using ArrayType = typename TypeTraits::ArrayType; using offset_type = typename Type::offset_type; @@ -310,14 +304,13 @@ void StringDataTransform(KernelContext* ctx, const ExecBatch& batch, } else { DCHECK(input.buffers[1]); // We must allocate new space for the offsets and shift the existing offsets - KERNEL_RETURN_IF_ERROR( - ctx, GetShiftedOffsets(ctx, *input.buffers[1], input.offset, - input.length, &out_arr->buffers[1])); + RETURN_NOT_OK(GetShiftedOffsets(ctx, *input.buffers[1], input.offset, + input.length, &out_arr->buffers[1])); } // Allocate space for output data int64_t data_nbytes = input_boxed.total_values_length(); - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(data_nbytes).Value(&out_arr->buffers[2])); + RETURN_NOT_OK(ctx->Allocate(data_nbytes).Value(&out_arr->buffers[2])); if (input.length > 0) { transform(input.buffers[2]->data() + input_boxed.value_offset(0), data_nbytes, out_arr->buffers[2]->mutable_data()); @@ -328,11 +321,13 @@ void StringDataTransform(KernelContext* ctx, const ExecBatch& batch, if (input.is_valid) { result->is_valid = true; int64_t data_nbytes = input.value->size(); - KERNEL_RETURN_IF_ERROR(ctx, ctx->Allocate(data_nbytes).Value(&result->value)); + RETURN_NOT_OK(ctx->Allocate(data_nbytes).Value(&result->value)); transform(input.value->data(), data_nbytes, result->value->mutable_data()); } out->value = result; } + + return Status::OK(); } void TransformAsciiUpper(const uint8_t* input, int64_t length, uint8_t* output) { @@ -341,8 +336,8 @@ void TransformAsciiUpper(const uint8_t* input, int64_t length, uint8_t* output) template struct AsciiUpper { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - StringDataTransform(ctx, batch, TransformAsciiUpper, out); + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return StringDataTransform(ctx, batch, TransformAsciiUpper, out); } }; @@ -352,8 +347,8 @@ void TransformAsciiLower(const uint8_t* input, int64_t length, uint8_t* output) template struct AsciiLower { - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - StringDataTransform(ctx, batch, TransformAsciiLower, out); + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return StringDataTransform(ctx, batch, TransformAsciiLower, out); } }; @@ -397,10 +392,9 @@ using MatchSubstringState = OptionsWrapper; template struct MatchSubstring { using offset_type = typename Type::offset_type; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // TODO Cache matcher across invocations (for regex compilation) - Matcher matcher(ctx, MatchSubstringState::Get(ctx)); - if (ctx->HasError()) return; + ARROW_ASSIGN_OR_RAISE(auto matcher, Matcher::Make(MatchSubstringState::Get(ctx))); StringBoolTransform( ctx, batch, [&matcher](const void* raw_offsets, const uint8_t* data, int64_t length, @@ -410,7 +404,7 @@ struct MatchSubstring { for (int64_t i = 0; i < length; ++i) { const char* current_data = reinterpret_cast(data + offsets[i]); int64_t current_length = offsets[i + 1] - offsets[i]; - if (matcher.Match(util::string_view(current_data, current_length))) { + if (matcher->Match(util::string_view(current_data, current_length))) { bitmap_writer.Set(); } bitmap_writer.Next(); @@ -418,6 +412,7 @@ struct MatchSubstring { bitmap_writer.Finish(); }, out); + return Status::OK(); } }; @@ -426,7 +421,12 @@ struct PlainSubstringMatcher { const MatchSubstringOptions& options_; std::vector prefix_table; - PlainSubstringMatcher(KernelContext* ctx, const MatchSubstringOptions& options) + static Result> Make( + const MatchSubstringOptions& options) { + return ::arrow::internal::make_unique(options); + } + + explicit PlainSubstringMatcher(const MatchSubstringOptions& options) : options_(options) { // Phase 1: Build the prefix table const auto pattern_length = options_.pattern.size(); @@ -444,7 +444,7 @@ struct PlainSubstringMatcher { } } - bool Match(util::string_view current) { + bool Match(util::string_view current) const { // Phase 2: Find the prefix in the data const auto pattern_length = options_.pattern.size(); int64_t pattern_pos = 0; @@ -472,12 +472,17 @@ struct RegexSubstringMatcher { const MatchSubstringOptions& options_; const RE2 regex_match_; - RegexSubstringMatcher(KernelContext* ctx, const MatchSubstringOptions& options) - : options_(options), regex_match_(options_.pattern, RE2::Quiet) { - KERNEL_RETURN_IF_ERROR(ctx, RegexStatus(regex_match_)); + static Result> Make( + const MatchSubstringOptions& options) { + auto matcher = ::arrow::internal::make_unique(options); + RETURN_NOT_OK(RegexStatus(matcher->regex_match_)); + return std::move(matcher); } - bool Match(util::string_view current) { + explicit RegexSubstringMatcher(const MatchSubstringOptions& options) + : options_(options), regex_match_(options_.pattern, RE2::Quiet) {} + + bool Match(util::string_view current) const { auto piece = re2::StringPiece(current.data(), current.length()); return re2::RE2::PartialMatch(piece, regex_match_); } @@ -661,8 +666,8 @@ static inline bool IsPrintableCharacterAscii(uint8_t ascii_character) { template struct CharacterPredicateUnicode { - static bool Call(KernelContext* ctx, const uint8_t* input, - size_t input_string_ncodeunits) { + static bool Call(KernelContext*, const uint8_t* input, size_t input_string_ncodeunits, + Status* st) { if (allow_empty && input_string_ncodeunits == 0) { return true; } @@ -673,7 +678,7 @@ struct CharacterPredicateUnicode { any |= Derived::PredicateCharacterAny(codepoint); return Derived::PredicateCharacterAll(codepoint); }))) { - ctx->SetStatus(Status::Invalid("Invalid UTF8 sequence in input")); + *st = Status::Invalid("Invalid UTF8 sequence in input"); return false; } return all & any; @@ -686,8 +691,8 @@ struct CharacterPredicateUnicode { template struct CharacterPredicateAscii { - static bool Call(KernelContext* ctx, const uint8_t* input, - size_t input_string_ncodeunits) { + static bool Call(KernelContext*, const uint8_t* input, size_t input_string_ncodeunits, + Status*) { if (allow_empty && input_string_ncodeunits == 0) { return true; } @@ -764,8 +769,8 @@ struct IsNumericUnicode : CharacterPredicateUnicode { #endif struct IsAscii { - static bool Call(KernelContext* ctx, const uint8_t* input, - size_t input_string_nascii_characters) { + static bool Call(KernelContext*, const uint8_t* input, + size_t input_string_nascii_characters, Status*) { return std::all_of(input, input + input_string_nascii_characters, IsAsciiCharacter); } @@ -826,8 +831,8 @@ struct IsSpaceAscii : CharacterPredicateAscii { #ifdef ARROW_WITH_UTF8PROC struct IsTitleUnicode { - static bool Call(KernelContext* ctx, const uint8_t* input, - size_t input_string_ncodeunits) { + static bool Call(KernelContext*, const uint8_t* input, size_t input_string_ncodeunits, + Status* st) { // rules: // * 1: lower case follows cased // * 2: upper case follows uncased @@ -854,7 +859,7 @@ struct IsTitleUnicode { return true; }); if (!ARROW_PREDICT_TRUE(status)) { - ctx->SetStatus(Status::Invalid("Invalid UTF8 sequence in input")); + *st = Status::Invalid("Invalid UTF8 sequence in input"); return false; } return rules_1_and_2 & rule_3; @@ -863,8 +868,8 @@ struct IsTitleUnicode { #endif struct IsTitleAscii { - static bool Call(KernelContext* ctx, const uint8_t* input, - size_t input_string_ncodeunits) { + static bool Call(KernelContext*, const uint8_t* input, size_t input_string_ncodeunits, + Status*) { // rules: // * 1: lower case follows cased // * 2: upper case follows uncased @@ -1001,15 +1006,15 @@ struct SplitBaseTransform { static Status CheckOptions(const Options& options) { return Status::OK(); } - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { Options options = State::Get(ctx); Derived splitter(options); // we make an instance to reuse the parts vectors - splitter.Split(ctx, batch, out); + return splitter.Split(ctx, batch, out); } - void Split(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Split(KernelContext* ctx, const ExecBatch& batch, Datum* out) { EnsureLookupTablesFilled(); // only needed for unicode - KERNEL_RETURN_IF_ERROR(ctx, Derived::CheckOptions(options)); + RETURN_NOT_OK(Derived::CheckOptions(options)); if (batch[0].kind() == Datum::ARRAY) { const ArrayData& input = *batch[0].array(); @@ -1017,9 +1022,9 @@ struct SplitBaseTransform { BuilderType builder(input.type, ctx->memory_pool()); // a slight overestimate of the data needed - KERNEL_RETURN_IF_ERROR(ctx, builder.ReserveData(input_boxed.total_values_length())); + RETURN_NOT_OK(builder.ReserveData(input_boxed.total_values_length())); // the minimum amount of strings needed - KERNEL_RETURN_IF_ERROR(ctx, builder.Resize(input.length)); + RETURN_NOT_OK(builder.Resize(input.length)); ArrayData* output_list = out->mutable_array(); // list offsets were preallocated @@ -1027,27 +1032,25 @@ struct SplitBaseTransform { DCHECK_NE(list_offsets, nullptr); // initial value *list_offsets++ = 0; - KERNEL_RETURN_IF_ERROR( - ctx, - VisitArrayDataInline( - input, - [&](util::string_view s) { - RETURN_NOT_OK(Split(s, &builder)); - if (ARROW_PREDICT_FALSE(builder.length() > - std::numeric_limits::max())) { - return Status::CapacityError("List offset does not fit into 32 bit"); - } - *list_offsets++ = static_cast(builder.length()); - return Status::OK(); - }, - [&]() { - // null value is already taken from input - *list_offsets++ = static_cast(builder.length()); - return Status::OK(); - })); + RETURN_NOT_OK(VisitArrayDataInline( + input, + [&](util::string_view s) { + RETURN_NOT_OK(Split(s, &builder)); + if (ARROW_PREDICT_FALSE(builder.length() > + std::numeric_limits::max())) { + return Status::CapacityError("List offset does not fit into 32 bit"); + } + *list_offsets++ = static_cast(builder.length()); + return Status::OK(); + }, + [&]() { + // null value is already taken from input + *list_offsets++ = static_cast(builder.length()); + return Status::OK(); + })); // assign list child data std::shared_ptr string_array; - KERNEL_RETURN_IF_ERROR(ctx, builder.Finish(&string_array)); + RETURN_NOT_OK(builder.Finish(&string_array)); output_list->child_data.push_back(string_array->data()); } else { @@ -1057,11 +1060,13 @@ struct SplitBaseTransform { result->is_valid = true; BuilderType builder(input.type, ctx->memory_pool()); util::string_view s(*input.value); - KERNEL_RETURN_IF_ERROR(ctx, Split(s, &builder)); - KERNEL_RETURN_IF_ERROR(ctx, builder.Finish(&result->value)); + RETURN_NOT_OK(Split(s, &builder)); + RETURN_NOT_OK(builder.Finish(&result->value)); } out->value = result; } + + return Status::OK(); } }; @@ -1314,64 +1319,68 @@ struct ReplaceSubString { using OffsetBuilder = TypedBufferBuilder; using State = OptionsWrapper; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // TODO Cache replacer across invocations (for regex compilation) - Replacer replacer{ctx, State::Get(ctx)}; - if (!ctx->HasError()) { - Replace(ctx, batch, &replacer, out); - } + ARROW_ASSIGN_OR_RAISE(auto replacer, Replacer::Make(State::Get(ctx))); + return Replace(ctx, batch, *replacer, out); } - static void Replace(KernelContext* ctx, const ExecBatch& batch, Replacer* replacer, - Datum* out) { + static Status Replace(KernelContext* ctx, const ExecBatch& batch, + const Replacer& replacer, Datum* out) { ValueDataBuilder value_data_builder(ctx->memory_pool()); OffsetBuilder offset_builder(ctx->memory_pool()); if (batch[0].kind() == Datum::ARRAY) { // We already know how many strings we have, so we can use Reserve/UnsafeAppend - KERNEL_RETURN_IF_ERROR(ctx, offset_builder.Reserve(batch[0].array()->length)); + RETURN_NOT_OK(offset_builder.Reserve(batch[0].array()->length)); offset_builder.UnsafeAppend(0); // offsets start at 0 const ArrayData& input = *batch[0].array(); - KERNEL_RETURN_IF_ERROR( - ctx, VisitArrayDataInline( - input, - [&](util::string_view s) { - RETURN_NOT_OK(replacer->ReplaceString(s, &value_data_builder)); - offset_builder.UnsafeAppend( - static_cast(value_data_builder.length())); - return Status::OK(); - }, - [&]() { - // offset for null value - offset_builder.UnsafeAppend( - static_cast(value_data_builder.length())); - return Status::OK(); - })); + RETURN_NOT_OK(VisitArrayDataInline( + input, + [&](util::string_view s) { + RETURN_NOT_OK(replacer.ReplaceString(s, &value_data_builder)); + offset_builder.UnsafeAppend( + static_cast(value_data_builder.length())); + return Status::OK(); + }, + [&]() { + // offset for null value + offset_builder.UnsafeAppend( + static_cast(value_data_builder.length())); + return Status::OK(); + })); ArrayData* output = out->mutable_array(); - KERNEL_RETURN_IF_ERROR(ctx, value_data_builder.Finish(&output->buffers[2])); - KERNEL_RETURN_IF_ERROR(ctx, offset_builder.Finish(&output->buffers[1])); + RETURN_NOT_OK(value_data_builder.Finish(&output->buffers[2])); + RETURN_NOT_OK(offset_builder.Finish(&output->buffers[1])); } else { const auto& input = checked_cast(*batch[0].scalar()); auto result = std::make_shared(); if (input.is_valid) { util::string_view s = static_cast(*input.value); - KERNEL_RETURN_IF_ERROR(ctx, replacer->ReplaceString(s, &value_data_builder)); - KERNEL_RETURN_IF_ERROR(ctx, value_data_builder.Finish(&result->value)); + RETURN_NOT_OK(replacer.ReplaceString(s, &value_data_builder)); + RETURN_NOT_OK(value_data_builder.Finish(&result->value)); result->is_valid = true; } out->value = result; } + + return Status::OK(); } }; struct PlainSubStringReplacer { const ReplaceSubstringOptions& options_; - PlainSubStringReplacer(KernelContext* ctx, const ReplaceSubstringOptions& options) + static Result> Make( + const ReplaceSubstringOptions& options) { + return arrow::internal::make_unique(options); + } + + explicit PlainSubStringReplacer(const ReplaceSubstringOptions& options) : options_(options) {} - Status ReplaceString(util::string_view s, TypedBufferBuilder* builder) { + Status ReplaceString(util::string_view s, TypedBufferBuilder* builder) const { const char* i = s.begin(); const char* end = s.end(); int64_t max_replacements = options_.max_replacements; @@ -1396,9 +1405,8 @@ struct PlainSubStringReplacer { } } // if we exited early due to max_replacements, add the trailing part - RETURN_NOT_OK(builder->Append(reinterpret_cast(i), - static_cast(end - i))); - return Status::OK(); + return builder->Append(reinterpret_cast(i), + static_cast(end - i)); } }; @@ -1408,31 +1416,38 @@ struct RegexSubStringReplacer { const RE2 regex_find_; const RE2 regex_replacement_; + static Result> Make( + const ReplaceSubstringOptions& options) { + auto replacer = arrow::internal::make_unique(options); + + RETURN_NOT_OK(RegexStatus(replacer->regex_find_)); + RETURN_NOT_OK(RegexStatus(replacer->regex_replacement_)); + + std::string replacement_error; + if (!replacer->regex_replacement_.CheckRewriteString(replacer->options_.replacement, + &replacement_error)) { + return Status::Invalid("Invalid replacement string: ", + std::move(replacement_error)); + } + + return std::move(replacer); + } + // Using RE2::FindAndConsume we can only find the pattern if it is a group, therefore // we have 2 regexes, one with () around it, one without. - RegexSubStringReplacer(KernelContext* ctx, const ReplaceSubstringOptions& options) + explicit RegexSubStringReplacer(const ReplaceSubstringOptions& options) : options_(options), regex_find_("(" + options_.pattern + ")", RE2::Quiet), - regex_replacement_(options_.pattern, RE2::Quiet) { - KERNEL_RETURN_IF_ERROR(ctx, RegexStatus(regex_find_)); - KERNEL_RETURN_IF_ERROR(ctx, RegexStatus(regex_replacement_)); - std::string replacement_error; - if (!regex_replacement_.CheckRewriteString(options_.replacement, - &replacement_error)) { - ctx->SetStatus( - Status::Invalid("Invalid replacement string: ", std::move(replacement_error))); - } - } + regex_replacement_(options_.pattern, RE2::Quiet) {} - Status ReplaceString(util::string_view s, TypedBufferBuilder* builder) { + Status ReplaceString(util::string_view s, TypedBufferBuilder* builder) const { re2::StringPiece replacement(options_.replacement); if (options_.max_replacements == -1) { std::string s_copy(s.to_string()); re2::RE2::GlobalReplace(&s_copy, regex_replacement_, replacement); - RETURN_NOT_OK(builder->Append(reinterpret_cast(s_copy.data()), - s_copy.length())); - return Status::OK(); + return builder->Append(reinterpret_cast(s_copy.data()), + s_copy.length()); } // Since RE2 does not have the concept of max_replacements, we have to do some work @@ -1467,9 +1482,8 @@ struct RegexSubStringReplacer { } } // If we exited early due to max_replacements, add the trailing part - RETURN_NOT_OK(builder->Append(reinterpret_cast(i), - static_cast(end - i))); - return Status::OK(); + return builder->Append(reinterpret_cast(i), + static_cast(end - i)); } }; #endif @@ -1598,21 +1612,20 @@ struct ExtractRegex : public ExtractRegexBase { using ExtractRegexBase::ExtractRegexBase; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { ExtractRegexOptions options = State::Get(ctx); - KERNEL_ASSIGN_OR_RAISE(auto data, ctx, ExtractRegexData::Make(options)); - ExtractRegex{data}.Extract(ctx, batch, out); + ARROW_ASSIGN_OR_RAISE(auto data, ExtractRegexData::Make(options)); + return ExtractRegex{data}.Extract(ctx, batch, out); } - void Extract(KernelContext* ctx, const ExecBatch& batch, Datum* out) { - KERNEL_ASSIGN_OR_RAISE(auto descr, ctx, - data.ResolveOutputType(batch.GetDescriptors())); + Status Extract(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + ARROW_ASSIGN_OR_RAISE(auto descr, data.ResolveOutputType(batch.GetDescriptors())); DCHECK_NE(descr.type, nullptr); const auto& type = descr.type; if (batch[0].kind() == Datum::ARRAY) { std::unique_ptr array_builder; - KERNEL_RETURN_IF_ERROR(ctx, MakeBuilder(ctx->memory_pool(), type, &array_builder)); + RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), type, &array_builder)); StructBuilder* struct_builder = checked_cast(array_builder.get()); std::vector field_builders; @@ -1639,11 +1652,10 @@ struct ExtractRegex : public ExtractRegexBase { } }; const ArrayData& input = *batch[0].array(); - KERNEL_RETURN_IF_ERROR(ctx, - VisitArrayDataInline(input, visit_value, visit_null)); + RETURN_NOT_OK(VisitArrayDataInline(input, visit_value, visit_null)); std::shared_ptr out_array; - KERNEL_RETURN_IF_ERROR(ctx, struct_builder->Finish(&out_array)); + RETURN_NOT_OK(struct_builder->Finish(&out_array)); *out = std::move(out_array); } else { const auto& input = checked_cast(*batch[0].scalar()); @@ -1660,6 +1672,8 @@ struct ExtractRegex : public ExtractRegexBase { } out->value = std::move(result); } + + return Status::OK(); } }; @@ -1707,12 +1721,11 @@ struct ParseStrptime { : parser(TimestampParser::MakeStrptime(options.format)), unit(options.unit) {} template - int64_t Call(KernelContext* ctx, util::string_view val) const { + int64_t Call(KernelContext*, util::string_view val, Status* st) const { int64_t result = 0; if (!(*parser)(val.data(), val.size(), unit, &result)) { - ctx->SetStatus(Status::Invalid("Failed to parse string: '", val, - "' as a scalar of type ", - TimestampType(unit).ToString())); + *st = Status::Invalid("Failed to parse string: '", val, "' as a scalar of type ", + TimestampType(unit).ToString()); } return result; } @@ -1722,7 +1735,7 @@ struct ParseStrptime { }; template -void StrptimeExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status StrptimeExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { applicator::ScalarUnaryNotNullStateful kernel{ ParseStrptime(StrptimeState::Get(ctx))}; return kernel.Exec(ctx, batch, out); @@ -1764,9 +1777,9 @@ struct UTF8TrimWhitespaceBase : StringTransform { *output_written = static_cast(end_trimmed - begin_trimmed); return true; } - void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { EnsureLookupTablesFilled(); - Base::Execute(ctx, batch, out); + return Base::Execute(ctx, batch, out); } }; @@ -1785,6 +1798,8 @@ struct UTF8RTrimWhitespace struct TrimStateUTF8 { TrimOptions options_; std::vector codepoints_; + Status status_ = Status::OK(); + explicit TrimStateUTF8(KernelContext* ctx, TrimOptions options) : options_(std::move(options)) { if (!ARROW_PREDICT_TRUE( @@ -1793,7 +1808,7 @@ struct TrimStateUTF8 { std::max(c + 1, static_cast(codepoints_.size()))); codepoints_.at(c) = true; }))) { - ctx->SetStatus(Status::Invalid("Invalid UTF8 sequence in input")); + status_ = Status::Invalid("Invalid UTF8 sequence in input"); } } }; @@ -1807,14 +1822,15 @@ struct UTF8TrimBase : StringTransform { explicit UTF8TrimBase(TrimStateUTF8 state) : state_(std::move(state)) {} - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { TrimStateUTF8 state = State::Get(ctx); - Derived(state).Execute(ctx, batch, out); + RETURN_NOT_OK(state.status_); + return Derived(state).Execute(ctx, batch, out); } - void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + Status Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) { EnsureLookupTablesFilled(); - Base::Execute(ctx, batch, out); + return Base::Execute(ctx, batch, out); } bool Transform(const uint8_t* input, offset_type input_string_ncodeunits, @@ -1843,6 +1859,7 @@ struct UTF8TrimBase : StringTransform { return true; } }; + template struct UTF8Trim : UTF8TrimBase> { using Base = UTF8TrimBase>; @@ -1911,9 +1928,9 @@ struct AsciiTrimBase : StringTransform { [&](char c) { characters_[static_cast(c)] = true; }); } - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { TrimOptions options = State::Get(ctx); - Derived(options).Execute(ctx, batch, out); + return Derived(options).Execute(ctx, batch, out); } bool Transform(const uint8_t* input, offset_type input_string_ncodeunits, @@ -2155,11 +2172,15 @@ void MakeUnaryStringUTF8TransformKernel(std::string name, FunctionRegistry* regi #endif -using StringPredicate = std::function; +// NOTE: Predicate should only populate 'status' with errors, +// leave it unmodified to indicate Status::OK() +using StringPredicate = + std::function; template -void ApplyPredicate(KernelContext* ctx, const ExecBatch& batch, StringPredicate predicate, - Datum* out) { +Status ApplyPredicate(KernelContext* ctx, const ExecBatch& batch, + StringPredicate predicate, Datum* out) { + Status st = Status::OK(); EnsureLookupTablesFilled(); if (batch[0].kind() == Datum::ARRAY) { const ArrayData& input = *batch[0].array(); @@ -2169,20 +2190,21 @@ void ApplyPredicate(KernelContext* ctx, const ExecBatch& batch, StringPredicate out_arr->buffers[1]->mutable_data(), out_arr->offset, input.length, [&]() -> bool { util::string_view val = input_it(); - return predicate(ctx, reinterpret_cast(val.data()), val.size()); + return predicate(ctx, reinterpret_cast(val.data()), val.size(), + &st); }); } else { const auto& input = checked_cast(*batch[0].scalar()); if (input.is_valid) { - bool boolean_result = - predicate(ctx, input.value->data(), static_cast(input.value->size())); - if (!ctx->status().ok()) { - // UTF decoding can lead to issues - return; + bool boolean_result = predicate(ctx, input.value->data(), + static_cast(input.value->size()), &st); + // UTF decoding can lead to issues + if (st.ok()) { + out->value = std::make_shared(boolean_result); } - out->value = std::make_shared(boolean_result); } } + return st; } template @@ -2190,10 +2212,10 @@ void AddUnaryStringPredicate(std::string name, FunctionRegistry* registry, const FunctionDoc* doc) { auto func = std::make_shared(name, Arity::Unary(), doc); auto exec_32 = [](KernelContext* ctx, const ExecBatch& batch, Datum* out) { - ApplyPredicate(ctx, batch, Predicate::Call, out); + return ApplyPredicate(ctx, batch, Predicate::Call, out); }; auto exec_64 = [](KernelContext* ctx, const ExecBatch& batch, Datum* out) { - ApplyPredicate(ctx, batch, Predicate::Call, out); + return ApplyPredicate(ctx, batch, Predicate::Call, out); }; DCHECK_OK(func->AddKernel({utf8()}, boolean(), std::move(exec_32))); DCHECK_OK(func->AddKernel({large_utf8()}, boolean(), std::move(exec_64))); diff --git a/cpp/src/arrow/compute/kernels/scalar_validity.cc b/cpp/src/arrow/compute/kernels/scalar_validity.cc index 1d399f322bf..ebb3dca0d1e 100644 --- a/cpp/src/arrow/compute/kernels/scalar_validity.cc +++ b/cpp/src/arrow/compute/kernels/scalar_validity.cc @@ -32,11 +32,12 @@ namespace internal { namespace { struct IsValidOperator { - static void Call(KernelContext* ctx, const Scalar& in, Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) { checked_cast(out)->value = in.is_valid; + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& arr, ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& arr, ArrayData* out) { DCHECK_EQ(out->offset, 0); DCHECK_LE(out->length, arr.length); if (arr.MayHaveNulls()) { @@ -48,37 +49,40 @@ struct IsValidOperator { arr.offset == 0 ? arr.buffers[0] : SliceBuffer(arr.buffers[0], arr.offset / 8, BitUtil::BytesForBits(out->length + out->offset)); - return; + return Status::OK(); } // Input has no nulls => output is entirely true. - KERNEL_ASSIGN_OR_RAISE(out->buffers[1], ctx, - ctx->AllocateBitmap(out->length + out->offset)); + ARROW_ASSIGN_OR_RAISE(out->buffers[1], + ctx->AllocateBitmap(out->length + out->offset)); BitUtil::SetBitsTo(out->buffers[1]->mutable_data(), out->offset, out->length, true); + return Status::OK(); } }; struct IsNullOperator { - static void Call(KernelContext* ctx, const Scalar& in, Scalar* out) { + static Status Call(KernelContext* ctx, const Scalar& in, Scalar* out) { checked_cast(out)->value = !in.is_valid; + return Status::OK(); } - static void Call(KernelContext* ctx, const ArrayData& arr, ArrayData* out) { + static Status Call(KernelContext* ctx, const ArrayData& arr, ArrayData* out) { if (arr.MayHaveNulls()) { // Input has nulls => output is the inverted null (validity) bitmap. InvertBitmap(arr.buffers[0]->data(), arr.offset, arr.length, out->buffers[1]->mutable_data(), out->offset); - return; + } else { + // Input has no nulls => output is entirely false. + BitUtil::SetBitsTo(out->buffers[1]->mutable_data(), out->offset, out->length, + false); } - - // Input has no nulls => output is entirely false. - BitUtil::SetBitsTo(out->buffers[1]->mutable_data(), out->offset, out->length, false); + return Status::OK(); } }; struct IsNanOperator { template - static constexpr OutType Call(KernelContext*, const InType& value) { + static constexpr OutType Call(KernelContext*, const InType& value, Status*) { return std::isnan(value); } }; @@ -116,7 +120,7 @@ std::shared_ptr MakeIsNanFunction(std::string name, return func; } -void IsValidExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status IsValidExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const Datum& arg0 = batch[0]; if (arg0.type()->id() == Type::NA) { auto false_value = std::make_shared(false); @@ -124,17 +128,17 @@ void IsValidExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { out->value = false_value; } else { std::shared_ptr false_values; - KERNEL_RETURN_IF_ERROR( - ctx, MakeArrayFromScalar(*false_value, out->length(), ctx->memory_pool()) - .Value(&false_values)); + RETURN_NOT_OK(MakeArrayFromScalar(*false_value, out->length(), ctx->memory_pool()) + .Value(&false_values)); out->value = false_values->data(); } + return Status::OK(); } else { - applicator::SimpleUnary(ctx, batch, out); + return applicator::SimpleUnary(ctx, batch, out); } } -void IsNullExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status IsNullExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const Datum& arg0 = batch[0]; if (arg0.type()->id() == Type::NA) { if (arg0.kind() == Datum::SCALAR) { @@ -145,8 +149,9 @@ void IsNullExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { BitUtil::SetBitsTo(out_arr->buffers[1]->mutable_data(), out_arr->offset, out_arr->length, true); } + return Status::OK(); } else { - applicator::SimpleUnary(ctx, batch, out); + return applicator::SimpleUnary(ctx, batch, out); } } diff --git a/cpp/src/arrow/compute/kernels/util_internal.cc b/cpp/src/arrow/compute/kernels/util_internal.cc index 1656ed2aaf3..846fa26baf2 100644 --- a/cpp/src/arrow/compute/kernels/util_internal.cc +++ b/cpp/src/arrow/compute/kernels/util_internal.cc @@ -59,24 +59,21 @@ PrimitiveArg GetPrimitiveArg(const ArrayData& arr) { ArrayKernelExec TrivialScalarUnaryAsArraysExec(ArrayKernelExec exec, NullHandling::type null_handling) { - return [=](KernelContext* ctx, const ExecBatch& batch, Datum* out) { + return [=](KernelContext* ctx, const ExecBatch& batch, Datum* out) -> Status { if (out->is_array()) { return exec(ctx, batch, out); } if (null_handling == NullHandling::INTERSECTION && !batch[0].scalar()->is_valid) { out->scalar()->is_valid = false; - return; + return Status::OK(); } - KERNEL_ASSIGN_OR_RAISE(Datum array_in, ctx, - MakeArrayFromScalar(*batch[0].scalar(), 1)); - - KERNEL_ASSIGN_OR_RAISE(Datum array_out, ctx, MakeArrayFromScalar(*out->scalar(), 1)); - - exec(ctx, ExecBatch{{std::move(array_in)}, 1}, &array_out); - - KERNEL_ASSIGN_OR_RAISE(*out, ctx, array_out.make_array()->GetScalar(0)); + ARROW_ASSIGN_OR_RAISE(Datum array_in, MakeArrayFromScalar(*batch[0].scalar(), 1)); + ARROW_ASSIGN_OR_RAISE(Datum array_out, MakeArrayFromScalar(*out->scalar(), 1)); + RETURN_NOT_OK(exec(ctx, ExecBatch{{std::move(array_in)}, 1}, &array_out)); + ARROW_ASSIGN_OR_RAISE(*out, array_out.make_array()->GetScalar(0)); + return Status::OK(); }; } diff --git a/cpp/src/arrow/compute/kernels/vector_hash.cc b/cpp/src/arrow/compute/kernels/vector_hash.cc index 0ed15702832..a68e78130f2 100644 --- a/cpp/src/arrow/compute/kernels/vector_hash.cc +++ b/cpp/src/arrow/compute/kernels/vector_hash.cc @@ -513,17 +513,19 @@ struct HashKernelTraits> { }; template -std::unique_ptr HashInitImpl(KernelContext* ctx, const KernelInitArgs& args) { +Result> HashInitImpl(KernelContext* ctx, + const KernelInitArgs& args) { using HashKernelType = typename HashKernelTraits::HashKernel; auto result = ::arrow::internal::make_unique( args.inputs[0].type, args.options, ctx->memory_pool()); - ctx->SetStatus(result->Reset()); + RETURN_NOT_OK(result->Reset()); return std::move(result); } template -std::unique_ptr HashInit(KernelContext* ctx, const KernelInitArgs& args) { - return std::move(HashInitImpl(ctx, args)); +Result> HashInit(KernelContext* ctx, + const KernelInitArgs& args) { + return HashInitImpl(ctx, args); } template @@ -574,10 +576,10 @@ KernelInit GetHashInit(Type::type type_id) { using DictionaryEncodeState = OptionsWrapper; template -std::unique_ptr DictionaryHashInit(KernelContext* ctx, - const KernelInitArgs& args) { +Result> DictionaryHashInit(KernelContext* ctx, + const KernelInitArgs& args) { const auto& dict_type = checked_cast(*args.inputs[0].type); - std::unique_ptr indices_hasher; + Result> indices_hasher; switch (dict_type.index_type()->id()) { case Type::INT8: indices_hasher = HashInitImpl(ctx, args); @@ -595,32 +597,37 @@ std::unique_ptr DictionaryHashInit(KernelContext* ctx, DCHECK(false) << "Unsupported dictionary index type"; break; } - return ::arrow::internal::make_unique(std::move(indices_hasher)); + RETURN_NOT_OK(indices_hasher); + return ::arrow::internal::make_unique( + std::move(indices_hasher.ValueOrDie())); } -void HashExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status HashExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { auto hash_impl = checked_cast(ctx->state()); - KERNEL_RETURN_IF_ERROR(ctx, hash_impl->Append(ctx, *batch[0].array())); - KERNEL_RETURN_IF_ERROR(ctx, hash_impl->Flush(out)); + RETURN_NOT_OK(hash_impl->Append(ctx, *batch[0].array())); + RETURN_NOT_OK(hash_impl->Flush(out)); + return Status::OK(); } -void UniqueFinalize(KernelContext* ctx, std::vector* out) { +Status UniqueFinalize(KernelContext* ctx, std::vector* out) { auto hash_impl = checked_cast(ctx->state()); std::shared_ptr uniques; - KERNEL_RETURN_IF_ERROR(ctx, hash_impl->GetDictionary(&uniques)); + RETURN_NOT_OK(hash_impl->GetDictionary(&uniques)); *out = {Datum(uniques)}; + return Status::OK(); } -void DictEncodeFinalize(KernelContext* ctx, std::vector* out) { +Status DictEncodeFinalize(KernelContext* ctx, std::vector* out) { auto hash_impl = checked_cast(ctx->state()); std::shared_ptr uniques; - KERNEL_RETURN_IF_ERROR(ctx, hash_impl->GetDictionary(&uniques)); + RETURN_NOT_OK(hash_impl->GetDictionary(&uniques)); auto dict_type = dictionary(int32(), uniques->type); auto dict = MakeArray(uniques); for (size_t i = 0; i < out->size(); ++i) { (*out)[i] = std::make_shared(dict_type, (*out)[i].make_array(), dict); } + return Status::OK(); } std::shared_ptr BoxValueCounts(const std::shared_ptr& uniques, @@ -631,33 +638,33 @@ std::shared_ptr BoxValueCounts(const std::shared_ptr& uniq return std::make_shared(data_type, uniques->length, children)->data(); } -void ValueCountsFinalize(KernelContext* ctx, std::vector* out) { +Status ValueCountsFinalize(KernelContext* ctx, std::vector* out) { auto hash_impl = checked_cast(ctx->state()); std::shared_ptr uniques; Datum value_counts; - KERNEL_RETURN_IF_ERROR(ctx, hash_impl->GetDictionary(&uniques)); - KERNEL_RETURN_IF_ERROR(ctx, hash_impl->FlushFinal(&value_counts)); + RETURN_NOT_OK(hash_impl->GetDictionary(&uniques)); + RETURN_NOT_OK(hash_impl->FlushFinal(&value_counts)); *out = {Datum(BoxValueCounts(uniques, value_counts.array()))}; + return Status::OK(); } -void UniqueFinalizeDictionary(KernelContext* ctx, std::vector* out) { - UniqueFinalize(ctx, out); - if (ctx->HasError()) { - return; - } +Status UniqueFinalizeDictionary(KernelContext* ctx, std::vector* out) { + RETURN_NOT_OK(UniqueFinalize(ctx, out)); auto hash = checked_cast(ctx->state()); (*out)[0].mutable_array()->dictionary = hash->dictionary(); + return Status::OK(); } -void ValueCountsFinalizeDictionary(KernelContext* ctx, std::vector* out) { +Status ValueCountsFinalizeDictionary(KernelContext* ctx, std::vector* out) { auto hash = checked_cast(ctx->state()); std::shared_ptr uniques; Datum value_counts; - KERNEL_RETURN_IF_ERROR(ctx, hash->GetDictionary(&uniques)); - KERNEL_RETURN_IF_ERROR(ctx, hash->FlushFinal(&value_counts)); + RETURN_NOT_OK(hash->GetDictionary(&uniques)); + RETURN_NOT_OK(hash->FlushFinal(&value_counts)); uniques->dictionary = hash->dictionary(); *out = {Datum(BoxValueCounts(uniques, value_counts.array()))}; + return Status::OK(); } ValueDescr DictEncodeOutput(KernelContext*, const std::vector& descrs) { diff --git a/cpp/src/arrow/compute/kernels/vector_nested.cc b/cpp/src/arrow/compute/kernels/vector_nested.cc index b7317e5bea0..b84640854ed 100644 --- a/cpp/src/arrow/compute/kernels/vector_nested.cc +++ b/cpp/src/arrow/compute/kernels/vector_nested.cc @@ -27,18 +27,15 @@ namespace internal { namespace { template -void ListFlatten(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ListFlatten(KernelContext* ctx, const ExecBatch& batch, Datum* out) { typename TypeTraits::ArrayType list_array(batch[0].array()); - Result> result = list_array.Flatten(ctx->memory_pool()); - if (!result.ok()) { - ctx->SetStatus(result.status()); - return; - } - out->value = (*result)->data(); + ARROW_ASSIGN_OR_RAISE(auto result, list_array.Flatten(ctx->memory_pool())); + out->value = result->data(); + return Status::OK(); } template -void ListParentIndices(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ListParentIndices(KernelContext* ctx, const ExecBatch& batch, Datum* out) { typename TypeTraits::ArrayType list(batch[0].array()); ArrayData* out_arr = out->mutable_array(); @@ -47,8 +44,8 @@ void ListParentIndices(KernelContext* ctx, const ExecBatch& batch, Datum* out) { out_arr->length = values_length; out_arr->null_count = 0; - KERNEL_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx, - ctx->Allocate(values_length * sizeof(offset_type))); + ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], + ctx->Allocate(values_length * sizeof(offset_type))); auto out_indices = reinterpret_cast(out_arr->buffers[1]->mutable_data()); for (int64_t i = 0; i < list.length(); ++i) { // Note: In most cases, null slots are empty, but when they are non-empty @@ -58,6 +55,7 @@ void ListParentIndices(KernelContext* ctx, const ExecBatch& batch, Datum* out) { *out_indices++ = static_cast(i); } } + return Status::OK(); } Result ValuesType(KernelContext*, const std::vector& args) { diff --git a/cpp/src/arrow/compute/kernels/vector_selection.cc b/cpp/src/arrow/compute/kernels/vector_selection.cc index 1c96f7699c6..fc7a78a2305 100644 --- a/cpp/src/arrow/compute/kernels/vector_selection.cc +++ b/cpp/src/arrow/compute/kernels/vector_selection.cc @@ -490,9 +490,9 @@ void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices, } } -void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (TakeState::Get(ctx).boundscheck) { - KERNEL_RETURN_IF_ERROR(ctx, CheckIndexBounds(*batch[1].array(), batch[0].length())); + RETURN_NOT_OK(CheckIndexBounds(*batch[1].array(), batch[0].length())); } PrimitiveArg values = GetPrimitiveArg(*batch[0].array()); @@ -504,23 +504,29 @@ void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // allocating the validity bitmap altogether and save time and space. A // streamlined PrimitiveTakeImpl would need to be written that skips all // interactions with the output validity bitmap, though. - KERNEL_RETURN_IF_ERROR(ctx, PreallocateData(ctx, indices.length, values.bit_width, - /*allocate_validity=*/true, out_arr)); + RETURN_NOT_OK(PreallocateData(ctx, indices.length, values.bit_width, + /*allocate_validity=*/true, out_arr)); switch (values.bit_width) { case 1: - return TakeIndexDispatch(values, indices, out_arr); + TakeIndexDispatch(values, indices, out_arr); + break; case 8: - return TakeIndexDispatch(values, indices, out_arr); + TakeIndexDispatch(values, indices, out_arr); + break; case 16: - return TakeIndexDispatch(values, indices, out_arr); + TakeIndexDispatch(values, indices, out_arr); + break; case 32: - return TakeIndexDispatch(values, indices, out_arr); + TakeIndexDispatch(values, indices, out_arr); + break; case 64: - return TakeIndexDispatch(values, indices, out_arr); + TakeIndexDispatch(values, indices, out_arr); + break; default: DCHECK(false) << "Invalid values byte width"; break; } + return Status::OK(); } // ---------------------------------------------------------------------- @@ -777,7 +783,7 @@ inline void PrimitiveFilterImpl::WriteNull() { BitUtil::ClearBit(out_data_, out_offset_ + out_position_++); } -void PrimitiveFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status PrimitiveFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { PrimitiveArg values = GetPrimitiveArg(*batch[0].array()); PrimitiveArg filter = GetPrimitiveArg(*batch[1].array()); FilterOptions::NullSelectionBehavior null_selection = @@ -802,29 +808,30 @@ void PrimitiveFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // validity bitmap. bool allocate_validity = values.null_count != 0 || filter.null_count != 0; - KERNEL_RETURN_IF_ERROR(ctx, PreallocateData(ctx, output_length, values.bit_width, - allocate_validity, out_arr)); + RETURN_NOT_OK( + PreallocateData(ctx, output_length, values.bit_width, allocate_validity, out_arr)); switch (values.bit_width) { case 1: - return PrimitiveFilterImpl(values, filter, null_selection, out_arr) - .Exec(); + PrimitiveFilterImpl(values, filter, null_selection, out_arr).Exec(); + break; case 8: - return PrimitiveFilterImpl(values, filter, null_selection, out_arr) - .Exec(); + PrimitiveFilterImpl(values, filter, null_selection, out_arr).Exec(); + break; case 16: - return PrimitiveFilterImpl(values, filter, null_selection, out_arr) - .Exec(); + PrimitiveFilterImpl(values, filter, null_selection, out_arr).Exec(); + break; case 32: - return PrimitiveFilterImpl(values, filter, null_selection, out_arr) - .Exec(); + PrimitiveFilterImpl(values, filter, null_selection, out_arr).Exec(); + break; case 64: - return PrimitiveFilterImpl(values, filter, null_selection, out_arr) - .Exec(); + PrimitiveFilterImpl(values, filter, null_selection, out_arr).Exec(); + break; default: DCHECK(false) << "Invalid values bit width"; break; } + return Status::OK(); } // ---------------------------------------------------------------------- @@ -1072,7 +1079,7 @@ Status BinaryFilterImpl(KernelContext* ctx, const ArrayData& values, #undef APPEND_RAW_DATA #undef APPEND_SINGLE_VALUE -void BinaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status BinaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { FilterOptions::NullSelectionBehavior null_selection = FilterState::Get(ctx).null_selection_behavior; @@ -1094,97 +1101,100 @@ void BinaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (values.null_count == 0 && filter.null_count == 0) { // Faster no-nulls case if (is_binary_like(type_id)) { - KERNEL_RETURN_IF_ERROR( - ctx, BinaryFilterNonNullImpl(ctx, values, filter, output_length, - null_selection, out_arr)); + RETURN_NOT_OK(BinaryFilterNonNullImpl( + ctx, values, filter, output_length, null_selection, out_arr)); } else if (is_large_binary_like(type_id)) { - KERNEL_RETURN_IF_ERROR( - ctx, BinaryFilterNonNullImpl( - ctx, values, filter, output_length, null_selection, out_arr)); + RETURN_NOT_OK(BinaryFilterNonNullImpl( + ctx, values, filter, output_length, null_selection, out_arr)); } else { DCHECK(false); } } else { // Output may have nulls - KERNEL_RETURN_IF_ERROR( - ctx, ctx->AllocateBitmap(output_length).Value(&out_arr->buffers[0])); + RETURN_NOT_OK(ctx->AllocateBitmap(output_length).Value(&out_arr->buffers[0])); if (is_binary_like(type_id)) { - KERNEL_RETURN_IF_ERROR( - ctx, BinaryFilterImpl(ctx, values, filter, output_length, - null_selection, out_arr)); - } else if (is_large_binary_like(type_id)) { - KERNEL_RETURN_IF_ERROR( - ctx, BinaryFilterImpl(ctx, values, filter, output_length, + RETURN_NOT_OK(BinaryFilterImpl(ctx, values, filter, output_length, null_selection, out_arr)); + } else if (is_large_binary_like(type_id)) { + RETURN_NOT_OK(BinaryFilterImpl(ctx, values, filter, output_length, + null_selection, out_arr)); } else { DCHECK(false); } } + + return Status::OK(); } // ---------------------------------------------------------------------- // Null take and filter -void NullTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status NullTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (TakeState::Get(ctx).boundscheck) { - KERNEL_RETURN_IF_ERROR(ctx, CheckIndexBounds(*batch[1].array(), batch[0].length())); + RETURN_NOT_OK(CheckIndexBounds(*batch[1].array(), batch[0].length())); } // batch.length doesn't take into account the take indices auto new_length = batch[1].array()->length; out->value = std::make_shared(new_length)->data(); + return Status::OK(); } -void NullFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status NullFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { int64_t output_length = GetFilterOutputSize( *batch[1].array(), FilterState::Get(ctx).null_selection_behavior); out->value = std::make_shared(output_length)->data(); + return Status::OK(); } // ---------------------------------------------------------------------- // Dictionary take and filter -void DictionaryTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status DictionaryTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DictionaryArray values(batch[0].array()); Datum result; - KERNEL_RETURN_IF_ERROR(ctx, Take(Datum(values.indices()), batch[1], TakeState::Get(ctx), - ctx->exec_context()) - .Value(&result)); + RETURN_NOT_OK( + Take(Datum(values.indices()), batch[1], TakeState::Get(ctx), ctx->exec_context()) + .Value(&result)); DictionaryArray taken_values(values.type(), result.make_array(), values.dictionary()); out->value = taken_values.data(); + return Status::OK(); } -void DictionaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status DictionaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { DictionaryArray dict_values(batch[0].array()); Datum result; - KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(dict_values.indices()), batch[1].array(), - FilterState::Get(ctx), ctx->exec_context()) - .Value(&result)); + RETURN_NOT_OK(Filter(Datum(dict_values.indices()), batch[1].array(), + FilterState::Get(ctx), ctx->exec_context()) + .Value(&result)); DictionaryArray filtered_values(dict_values.type(), result.make_array(), dict_values.dictionary()); out->value = filtered_values.data(); + return Status::OK(); } // ---------------------------------------------------------------------- // Extension take and filter -void ExtensionTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ExtensionTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) { ExtensionArray values(batch[0].array()); Datum result; - KERNEL_RETURN_IF_ERROR(ctx, Take(Datum(values.storage()), batch[1], TakeState::Get(ctx), - ctx->exec_context()) - .Value(&result)); + RETURN_NOT_OK( + Take(Datum(values.storage()), batch[1], TakeState::Get(ctx), ctx->exec_context()) + .Value(&result)); ExtensionArray taken_values(values.type(), result.make_array()); out->value = taken_values.data(); + return Status::OK(); } -void ExtensionFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status ExtensionFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { ExtensionArray ext_values(batch[0].array()); Datum result; - KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(ext_values.storage()), batch[1].array(), - FilterState::Get(ctx), ctx->exec_context()) - .Value(&result)); + RETURN_NOT_OK(Filter(Datum(ext_values.storage()), batch[1].array(), + FilterState::Get(ctx), ctx->exec_context()) + .Value(&result)); ExtensionArray filtered_values(ext_values.type(), result.make_array()); out->value = filtered_values.data(); + return Status::OK(); } // ---------------------------------------------------------------------- @@ -1742,20 +1752,20 @@ struct StructImpl : public Selection { } }; -void StructFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status StructFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // Transform filter to selection indices and then use Take. std::shared_ptr indices; - KERNEL_RETURN_IF_ERROR( - ctx, - GetTakeIndices(*batch[1].array(), FilterState::Get(ctx).null_selection_behavior, - ctx->memory_pool()) - .Value(&indices)); + RETURN_NOT_OK(GetTakeIndices(*batch[1].array(), + FilterState::Get(ctx).null_selection_behavior, + ctx->memory_pool()) + .Value(&indices)); Datum result; - KERNEL_RETURN_IF_ERROR(ctx, Take(batch[0], Datum(indices), TakeOptions::NoBoundsCheck(), - ctx->exec_context()) - .Value(&result)); + RETURN_NOT_OK( + Take(batch[0], Datum(indices), TakeOptions::NoBoundsCheck(), ctx->exec_context()) + .Value(&result)); out->value = result.array(); + return Status::OK(); } #undef LIFT_BASE_MEMBERS @@ -2064,21 +2074,21 @@ class TakeMetaFunction : public MetaFunction { // ---------------------------------------------------------------------- template -void FilterExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status FilterExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { // TODO: where are the values and filter length equality checked? int64_t output_length = GetFilterOutputSize( *batch[1].array(), FilterState::Get(ctx).null_selection_behavior); Impl kernel(ctx, batch, output_length, out); - KERNEL_RETURN_IF_ERROR(ctx, kernel.ExecFilter()); + return kernel.ExecFilter(); } template -void TakeExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { +Status TakeExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { if (TakeState::Get(ctx).boundscheck) { - KERNEL_RETURN_IF_ERROR(ctx, CheckIndexBounds(*batch[1].array(), batch[0].length())); + RETURN_NOT_OK(CheckIndexBounds(*batch[1].array(), batch[0].length())); } Impl kernel(ctx, batch, /*output_length=*/batch[1].length(), out); - KERNEL_RETURN_IF_ERROR(ctx, kernel.ExecTake()); + return kernel.ExecTake(); } struct SelectionKernelDescr { diff --git a/cpp/src/arrow/compute/kernels/vector_sort.cc b/cpp/src/arrow/compute/kernels/vector_sort.cc index 8593613c8f5..6c425d65550 100644 --- a/cpp/src/arrow/compute/kernels/vector_sort.cc +++ b/cpp/src/arrow/compute/kernels/vector_sort.cc @@ -322,27 +322,25 @@ template struct PartitionNthToIndices { using ArrayType = typename TypeTraits::ArrayType; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { using GetView = GetViewType; if (ctx->state() == nullptr) { - ctx->SetStatus(Status::Invalid("NthToIndices requires PartitionNthOptions")); - return; + return Status::Invalid("NthToIndices requires PartitionNthOptions"); } ArrayType arr(batch[0].array()); int64_t pivot = PartitionNthToIndicesState::Get(ctx).pivot; if (pivot > arr.length()) { - ctx->SetStatus(Status::IndexError("NthToIndices index out of bound")); - return; + return Status::IndexError("NthToIndices index out of bound"); } ArrayData* out_arr = out->mutable_array(); uint64_t* out_begin = out_arr->GetMutableValues(1); uint64_t* out_end = out_begin + arr.length(); std::iota(out_begin, out_end, 0); if (pivot == arr.length()) { - return; + return Status::OK(); } auto nulls_begin = PartitionNulls(out_begin, out_end, arr, 0); @@ -355,6 +353,7 @@ struct PartitionNthToIndices { return lval < rval; }); } + return Status::OK(); } }; @@ -559,7 +558,7 @@ using ArraySortIndicesState = internal::OptionsWrapper; template struct ArraySortIndices { using ArrayType = typename TypeTraits::ArrayType; - static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { + static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) { const auto& options = ArraySortIndicesState::Get(ctx); ArrayType arr(batch[0].array()); @@ -570,6 +569,8 @@ struct ArraySortIndices { ArraySorter sorter; sorter.impl.Sort(out_begin, out_end, arr, 0, options); + + return Status::OK(); } }; diff --git a/cpp/src/arrow/dataset/expression.cc b/cpp/src/arrow/dataset/expression.cc index 627477b3038..cc126fcc7fb 100644 --- a/cpp/src/arrow/dataset/expression.cc +++ b/cpp/src/arrow/dataset/expression.cc @@ -427,10 +427,10 @@ Result BindNonRecursive(Expression::Call call, bool insert_implicit_ compute::KernelContext kernel_context(exec_context); if (call.kernel->init) { - call.kernel_state = - call.kernel->init(&kernel_context, {call.kernel, descrs, call.options.get()}); + ARROW_ASSIGN_OR_RAISE( + call.kernel_state, + call.kernel->init(&kernel_context, {call.kernel, descrs, call.options.get()})); - RETURN_NOT_OK(kernel_context.status()); kernel_context.SetState(call.kernel_state.get()); }