Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions cpp/src/arrow/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -363,6 +363,7 @@ if(ARROW_COMPUTE)
compute/kernels/scalar_set_lookup.cc
compute/kernels/scalar_string.cc
compute/kernels/scalar_validity.cc
compute/kernels/scalar_fill_null.cc
compute/kernels/util_internal.cc
compute/kernels/vector_hash.cc
compute/kernels/vector_nested.cc
Expand Down
4 changes: 4 additions & 0 deletions cpp/src/arrow/compute/api_scalar.cc
Original file line number Diff line number Diff line change
Expand Up @@ -126,5 +126,9 @@ Result<Datum> Compare(const Datum& left, const Datum& right, CompareOptions opti
SCALAR_EAGER_UNARY(IsValid, "is_valid")
SCALAR_EAGER_UNARY(IsNull, "is_null")

Result<Datum> FillNull(const Datum& values, const Datum& fill_value, ExecContext* ctx) {
return CallFunction("fill_null", {values, fill_value}, ctx);
}

} // namespace compute
} // namespace arrow
15 changes: 15 additions & 0 deletions cpp/src/arrow/compute/api_scalar.h
Original file line number Diff line number Diff line change
Expand Up @@ -259,6 +259,21 @@ Result<Datum> IsValid(const Datum& values, ExecContext* ctx = NULLPTR);
ARROW_EXPORT
Result<Datum> IsNull(const Datum& values, ExecContext* ctx = NULLPTR);

/// \brief FillNull replaces each null element in `values`
/// with `fill_value`
///
/// \param[in] values input to examine for nullity
/// \param[in] fill_value scalar
/// \param[in] ctx the function execution context, optional
///
/// \return the resulting datum
///
/// \since 1.0.0
/// \note API not yet finalized
ARROW_EXPORT
Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
ExecContext* ctx = NULLPTR);

// ----------------------------------------------------------------------
// String functions

Expand Down
1 change: 1 addition & 0 deletions cpp/src/arrow/compute/kernels/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ add_arrow_compute_test(scalar_test
scalar_set_lookup_test.cc
scalar_string_test.cc
scalar_validity_test.cc
scalar_fill_null_test.cc
test_util.cc)

add_arrow_benchmark(scalar_arithmetic_benchmark PREFIX "arrow-compute")
Expand Down
40 changes: 40 additions & 0 deletions cpp/src/arrow/compute/kernels/codegen_internal.h
Original file line number Diff line number Diff line change
Expand Up @@ -852,6 +852,46 @@ ArrayKernelExec GenerateSignedInteger(detail::GetTypeId get_id) {
}
}

// Generate a kernel given a templated functor. Only a single template is
// instantiated for each bit width, and the functor is expected to treat types
// of the same bit width the same without utilizing any type-specific behavior
// (e.g. int64 should be handled equivalent to uint64 or double -- all 64
// bits).
//
// See "Numeric" above for description of the generator functor
template <template <typename...> class Generator>
ArrayKernelExec GenerateTypeAgnosticPrimitive(detail::GetTypeId get_id) {
switch (get_id.id) {
case Type::NA:
return Generator<NullType>::Exec;
case Type::BOOL:
return Generator<BooleanType>::Exec;
case Type::UINT8:
case Type::INT8:
return Generator<UInt8Type>::Exec;
case Type::UINT16:
case Type::INT16:
return Generator<UInt16Type>::Exec;
case Type::UINT32:
case Type::INT32:
case Type::FLOAT:
case Type::DATE32:
case Type::TIME32:
return Generator<UInt32Type>::Exec;
case Type::UINT64:
case Type::INT64:
case Type::DOUBLE:
case Type::DATE64:
case Type::TIMESTAMP:
case Type::TIME64:
case Type::DURATION:
return Generator<UInt64Type>::Exec;
default:
DCHECK(false);
return ExecFail;
}
}

// Generate a kernel given a templated functor for base binary types. Generates
// a single kernel for binary/string and large binary / large string. If your
// kernel implementation needs access to the specific type at compile time,
Expand Down
168 changes: 168 additions & 0 deletions cpp/src/arrow/compute/kernels/scalar_fill_null.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,168 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#include <algorithm>
#include <cstring>

#include "arrow/compute/kernels/common.h"
#include "arrow/scalar.h"
#include "arrow/util/bit_block_counter.h"
#include "arrow/util/bit_util.h"
#include "arrow/util/bitmap_ops.h"

namespace arrow {

using internal::BitBlockCount;
using internal::BitBlockCounter;

namespace compute {
namespace internal {

namespace {

template <typename Type, typename Enable = void>
struct FillNullFunctor {};

template <typename Type>
struct FillNullFunctor<Type, enable_if_t<is_number_type<Type>::value>> {
using T = typename TypeTraits<Type>::CType;

static void 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();

// Ensure the kernel is configured properly to have no validity bitmap /
// null count 0 unless we explicitly propagate it below.
DCHECK(output->buffers[0] == nullptr);

T value = UnboxScalar<Type>::Unbox(fill_value);
if (data.MayHaveNulls() != 0 && fill_value.is_valid) {
KERNEL_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf, ctx,
ctx->Allocate(data.length * sizeof(T)));

const uint8_t* is_valid = data.buffers[0]->data();
const T* in_values = data.GetValues<T>(1);
T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
int64_t offset = data.offset;
BitBlockCounter bit_counter(is_valid, data.offset, data.length);
while (offset < data.offset + data.length) {
BitBlockCount block = bit_counter.NextWord();
if (block.AllSet()) {
// Block all not null
std::memcpy(out_values, in_values, block.length * sizeof(T));
} else if (block.NoneSet()) {
// Block all null
std::fill(out_values, out_values + block.length, value);
} else {
for (int64_t i = 0; i < block.length; ++i) {
out_values[i] = BitUtil::GetBit(is_valid, offset + i) ? in_values[i] : value;
}
}
offset += block.length;
out_values += block.length;
in_values += block.length;
}
output->buffers[1] = out_buf;
} else {
*output = data;
}
}
};

template <typename Type>
struct FillNullFunctor<Type, enable_if_t<is_boolean_type<Type>::value>> {
static void 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<BooleanType>::Unbox(fill_value);
if (data.MayHaveNulls() != 0 && fill_value.is_valid) {
KERNEL_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf, ctx,
ctx->AllocateBitmap(data.length));

const uint8_t* is_valid = data.buffers[0]->data();
const uint8_t* data_bitmap = data.buffers[1]->data();
uint8_t* out_bitmap = out_buf->mutable_data();

int64_t offset = data.offset;
BitBlockCounter bit_counter(is_valid, data.offset, data.length);
while (offset < data.offset + data.length) {
BitBlockCount block = bit_counter.NextWord();
if (block.AllSet()) {
// Block all not null
::arrow::internal::CopyBitmap(data_bitmap, data.offset, block.length,
out_bitmap, offset);
} else if (block.NoneSet()) {
// Block all null
BitUtil::SetBitsTo(out_bitmap, offset, block.length, value);
} else {
for (int64_t i = 0; i < block.length; ++i) {
BitUtil::SetBitTo(out_bitmap, offset + i,
BitUtil::GetBit(is_valid, offset + i)
? BitUtil::GetBit(data_bitmap, offset + i)
: value);
}
}
offset += block.length;
}
output->buffers[1] = out_buf;
} else {
*output = data;
}
}
};

template <typename Type>
struct FillNullFunctor<Type, enable_if_t<is_null_type<Type>::value>> {
static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
// Nothing preallocated, so we assign into the output
*out->mutable_array() = *batch[0].array();
}
};

void AddBasicFillNullKernels(ScalarKernel kernel, ScalarFunction* func) {
auto AddKernels = [&](const std::vector<std::shared_ptr<DataType>>& types) {
for (const std::shared_ptr<DataType>& ty : types) {
kernel.signature =
KernelSignature::Make({InputType::Array(ty), InputType::Scalar(ty)}, ty);
kernel.exec = GenerateTypeAgnosticPrimitive<FillNullFunctor>(*ty);
DCHECK_OK(func->AddKernel(kernel));
}
};
AddKernels(NumericTypes());
AddKernels(TemporalTypes());
AddKernels({boolean(), null()});
}

} // namespace

void RegisterScalarFillNull(FunctionRegistry* registry) {
{
ScalarKernel fill_null_base;
fill_null_base.null_handling = NullHandling::COMPUTED_NO_PREALLOCATE;
fill_null_base.mem_allocation = MemAllocation::NO_PREALLOCATE;
auto fill_null = std::make_shared<ScalarFunction>("fill_null", Arity::Binary());
AddBasicFillNullKernels(fill_null_base, fill_null.get());
DCHECK_OK(registry->AddFunction(fill_null));
}
}

} // namespace internal
} // namespace compute
} // namespace arrow
109 changes: 109 additions & 0 deletions cpp/src/arrow/compute/kernels/scalar_fill_null_test.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#include <memory>
#include <string>

#include <gtest/gtest.h>

#include "arrow/array/array_base.h"
#include "arrow/compute/api.h"
#include "arrow/result.h"
#include "arrow/scalar.h"
#include "arrow/testing/gtest_compat.h"
#include "arrow/testing/gtest_util.h"
#include "arrow/type.h"
#include "arrow/type_traits.h"

namespace arrow {
namespace compute {

void CheckFillNull(const std::shared_ptr<DataType>& type, const std::string& in_values,
const Datum fill_value, const std::string& out_values) {
std::shared_ptr<Array> input = ArrayFromJSON(type, in_values);
std::shared_ptr<Array> expected = ArrayFromJSON(type, out_values);

ASSERT_OK_AND_ASSIGN(Datum datum_out, FillNull(input, fill_value));
std::shared_ptr<Array> result = datum_out.make_array();
ASSERT_OK(result->ValidateFull());
AssertArraysEqual(*expected, *result, /*verbose=*/true);
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use the ArrayFromJSON functions instead for specifying the test cases?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


class TestFillNullKernel : public ::testing::Test {};

template <typename Type>
class TestFillNullPrimitive : public ::testing::Test {};

typedef ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type, Int32Type,
UInt32Type, Int64Type, UInt64Type, FloatType, DoubleType,
Date32Type, Date64Type>
PrimitiveTypes;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are declared elsewhere I think?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see them declared a couple of times in the code base as PrimitiveDictionaries eg in scalar_set_lookup_test.cc - should I use that instead?


TEST_F(TestFillNullKernel, FillNullInvalidScalar) {
auto scalar = std::make_shared<Int8Scalar>(3);
scalar->is_valid = false;
CheckFillNull(int8(), "[1, null, 3, 2]", Datum(scalar), "[1, null, 3, 2]");
}

TYPED_TEST_SUITE(TestFillNullPrimitive, PrimitiveTypes);

TYPED_TEST(TestFillNullPrimitive, FillNull) {
using T = typename TypeParam::c_type;
using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
auto type = TypeTraits<TypeParam>::type_singleton();
auto scalar = std::make_shared<ScalarType>(static_cast<T>(5));
// No Nulls
CheckFillNull(type, "[2, 4, 7, 9]", Datum(scalar), "[2, 4, 7, 9]");
// Some Null
CheckFillNull(type, "[null, 4, null, 8]", Datum(scalar), "[5, 4, 5, 8]");
// Empty Array
CheckFillNull(type, "[]", Datum(scalar), "[]");
}

TEST_F(TestFillNullKernel, FillNullNull) {
auto datum = Datum(std::make_shared<NullScalar>());
CheckFillNull(null(), "[null, null, null, null]", datum, "[null, null, null, null]");
}

TEST_F(TestFillNullKernel, FillNullBoolean) {
auto scalar1 = std::make_shared<BooleanScalar>(false);
auto scalar2 = std::make_shared<BooleanScalar>(true);
// no nulls
CheckFillNull(boolean(), "[true, false, true, false]", Datum(scalar1),
"[true, false, true, false]");
// some nulls
CheckFillNull(boolean(), "[true, false, false, null]", Datum(scalar1),
"[true, false, false, false]");
CheckFillNull(boolean(), "[true, null, false, null]", Datum(scalar2),
"[true, true, false, true]");
}

TEST_F(TestFillNullKernel, FillNullTimeStamp) {
auto time32_type = time32(TimeUnit::SECOND);
auto time64_type = time64(TimeUnit::NANO);
auto scalar1 = std::make_shared<Time32Scalar>(5, time32_type);
auto scalar2 = std::make_shared<Time64Scalar>(6, time64_type);
// no nulls
CheckFillNull(time32_type, "[2, 1, 6, 9]", Datum(scalar1), "[2, 1, 6, 9]");
CheckFillNull(time64_type, "[2, 1, 6, 9]", Datum(scalar2), "[2, 1, 6, 9]");
// some nulls
CheckFillNull(time32_type, "[2, 1, 6, null]", Datum(scalar1), "[2, 1, 6, 5]");
CheckFillNull(time64_type, "[2, 1, 6, null]", Datum(scalar2), "[2, 1, 6, 6]");
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The behavior of the kernel when passing a scalar with is_valid=false is not validated (and probably yield incorrect results)

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have included a test case

} // namespace compute
} // namespace arrow
1 change: 1 addition & 0 deletions cpp/src/arrow/compute/registry.cc
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,7 @@ static std::unique_ptr<FunctionRegistry> CreateBuiltInRegistry() {
RegisterScalarSetLookup(registry.get());
RegisterScalarStringAscii(registry.get());
RegisterScalarValidity(registry.get());
RegisterScalarFillNull(registry.get());

// Aggregate functions
RegisterScalarAggregateBasic(registry.get());
Expand Down
1 change: 1 addition & 0 deletions cpp/src/arrow/compute/registry_internal.h
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ void RegisterScalarNested(FunctionRegistry* registry);
void RegisterScalarSetLookup(FunctionRegistry* registry);
void RegisterScalarStringAscii(FunctionRegistry* registry);
void RegisterScalarValidity(FunctionRegistry* registry);
void RegisterScalarFillNull(FunctionRegistry* registry);

// Vector functions
void RegisterVectorHash(FunctionRegistry* registry);
Expand Down