You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2021/06/30 09:21:02 UTC

[arrow] branch master updated: ARROW-13072: [C++] Add bit-wise arithmetic kernels

This is an automated email from the ASF dual-hosted git repository.

apitrou pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new e9fa304  ARROW-13072: [C++] Add bit-wise arithmetic kernels
e9fa304 is described below

commit e9fa30406215b76ed6c885302fbfe6075c47badf
Author: David Li <li...@gmail.com>
AuthorDate: Wed Jun 30 11:19:58 2021 +0200

    ARROW-13072: [C++] Add bit-wise arithmetic kernels
    
    Closes #10530 from lidavidm/arrow-13072
    
    Lead-authored-by: David Li <li...@gmail.com>
    Co-authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 cpp/src/arrow/compute/api_scalar.cc                |   2 +
 cpp/src/arrow/compute/api_scalar.h                 |  27 +++
 cpp/src/arrow/compute/kernels/scalar_arithmetic.cc | 259 +++++++++++++++++++++
 .../compute/kernels/scalar_arithmetic_test.cc      | 204 ++++++++++++++++
 docs/source/cpp/compute.rst                        |  63 +++--
 docs/source/python/api/compute.rst                 |  14 ++
 6 files changed, 551 insertions(+), 18 deletions(-)

diff --git a/cpp/src/arrow/compute/api_scalar.cc b/cpp/src/arrow/compute/api_scalar.cc
index db1cac2..f005e70 100644
--- a/cpp/src/arrow/compute/api_scalar.cc
+++ b/cpp/src/arrow/compute/api_scalar.cc
@@ -62,6 +62,8 @@ SCALAR_ARITHMETIC_BINARY(Subtract, "subtract", "subtract_checked")
 SCALAR_ARITHMETIC_BINARY(Multiply, "multiply", "multiply_checked")
 SCALAR_ARITHMETIC_BINARY(Divide, "divide", "divide_checked")
 SCALAR_ARITHMETIC_BINARY(Power, "power", "power_checked")
+SCALAR_ARITHMETIC_BINARY(ShiftLeft, "shift_left", "shift_left_checked")
+SCALAR_ARITHMETIC_BINARY(ShiftRight, "shift_right", "shift_right_checked")
 
 Result<Datum> MaxElementWise(const std::vector<Datum>& args,
                              ElementWiseAggregateOptions options, ExecContext* ctx) {
diff --git a/cpp/src/arrow/compute/api_scalar.h b/cpp/src/arrow/compute/api_scalar.h
index 5c83dcb..b101325 100644
--- a/cpp/src/arrow/compute/api_scalar.h
+++ b/cpp/src/arrow/compute/api_scalar.h
@@ -308,6 +308,33 @@ Result<Datum> Power(const Datum& left, const Datum& right,
                     ArithmeticOptions options = ArithmeticOptions(),
                     ExecContext* ctx = NULLPTR);
 
+/// \brief Left shift the left array by the right array. Array values must be the
+/// same length. If either operand is null, the result will be null.
+///
+/// \param[in] left the value to shift
+/// \param[in] right the value to shift by
+/// \param[in] options arithmetic options (enable/disable overflow checking), optional
+/// \param[in] ctx the function execution context, optional
+/// \return the elementwise left value shifted left by the right value
+ARROW_EXPORT
+Result<Datum> ShiftLeft(const Datum& left, const Datum& right,
+                        ArithmeticOptions options = ArithmeticOptions(),
+                        ExecContext* ctx = NULLPTR);
+
+/// \brief Right shift the left array by the right array. Array values must be the
+/// same length. If either operand is null, the result will be null. Performs a
+/// logical shift for unsigned values, and an arithmetic shift for signed values.
+///
+/// \param[in] left the value to shift
+/// \param[in] right the value to shift by
+/// \param[in] options arithmetic options (enable/disable overflow checking), optional
+/// \param[in] ctx the function execution context, optional
+/// \return the elementwise left value shifted right by the right value
+ARROW_EXPORT
+Result<Datum> ShiftRight(const Datum& left, const Datum& right,
+                         ArithmeticOptions options = ArithmeticOptions(),
+                         ExecContext* ctx = NULLPTR);
+
 /// \brief Find the element-wise maximum of any number of arrays or scalars.
 /// Array values must be the same length.
 ///
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
index f51484e..ef9ef78 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
@@ -454,6 +454,106 @@ struct PowerChecked {
   }
 };
 
+// Bitwise operations
+
+struct BitWiseNot {
+  template <typename T, typename Arg>
+  static T Call(KernelContext*, Arg arg, Status*) {
+    return ~arg;
+  }
+};
+
+struct BitWiseAnd {
+  template <typename T, typename Arg0, typename Arg1>
+  static T Call(KernelContext*, Arg0 lhs, Arg1 rhs, Status*) {
+    return lhs & rhs;
+  }
+};
+
+struct BitWiseOr {
+  template <typename T, typename Arg0, typename Arg1>
+  static T Call(KernelContext*, Arg0 lhs, Arg1 rhs, Status*) {
+    return lhs | rhs;
+  }
+};
+
+struct BitWiseXor {
+  template <typename T, typename Arg0, typename Arg1>
+  static T Call(KernelContext*, Arg0 lhs, Arg1 rhs, Status*) {
+    return lhs ^ rhs;
+  }
+};
+
+struct ShiftLeft {
+  template <typename T, typename Arg0, typename Arg1>
+  static T Call(KernelContext*, Arg0 lhs, Arg1 rhs, Status*) {
+    using Unsigned = typename std::make_unsigned<Arg0>::type;
+    static_assert(std::is_same<T, Arg0>::value, "");
+    if (ARROW_PREDICT_FALSE(rhs < 0 || rhs >= std::numeric_limits<Arg0>::digits)) {
+      return lhs;
+    }
+    return static_cast<T>(static_cast<Unsigned>(lhs) << static_cast<Unsigned>(rhs));
+  }
+};
+
+// See SEI CERT C Coding Standard rule INT34-C
+struct ShiftLeftChecked {
+  template <typename T, typename Arg0, typename Arg1>
+  static enable_if_unsigned_integer<T> Call(KernelContext*, Arg0 lhs, Arg1 rhs,
+                                            Status* st) {
+    static_assert(std::is_same<T, Arg0>::value, "");
+    if (ARROW_PREDICT_FALSE(rhs < 0 || rhs >= std::numeric_limits<Arg0>::digits)) {
+      *st = Status::Invalid("shift amount must be >= 0 and less than precision of type");
+      return lhs;
+    }
+    return lhs << rhs;
+  }
+
+  template <typename T, typename Arg0, typename Arg1>
+  static enable_if_signed_integer<T> Call(KernelContext*, Arg0 lhs, Arg1 rhs,
+                                          Status* st) {
+    using Unsigned = typename std::make_unsigned<Arg0>::type;
+    static_assert(std::is_same<T, Arg0>::value, "");
+    if (ARROW_PREDICT_FALSE(rhs < 0 || rhs >= std::numeric_limits<Arg0>::digits)) {
+      *st = Status::Invalid("shift amount must be >= 0 and less than precision of type");
+      return lhs;
+    }
+    // In C/C++ left shift of a negative number is undefined (C++11 standard 5.8.2)
+    // Mimic Java/etc. and treat left shift as based on two's complement representation
+    // Assumes two's complement machine
+    return static_cast<T>(static_cast<Unsigned>(lhs) << static_cast<Unsigned>(rhs));
+  }
+};
+
+struct ShiftRight {
+  template <typename T, typename Arg0, typename Arg1>
+  static T Call(KernelContext*, Arg0 lhs, Arg1 rhs, Status*) {
+    static_assert(std::is_same<T, Arg0>::value, "");
+    // Logical right shift when Arg0 is unsigned
+    // Arithmetic otherwise (this is implementation-defined but GCC and MSVC document this
+    // as arithmetic right shift)
+    // https://gcc.gnu.org/onlinedocs/gcc/Integers-implementation.html#Integers-implementation
+    // https://docs.microsoft.com/en-us/cpp/cpp/left-shift-and-right-shift-operators-input-and-output?view=msvc-160
+    // Clang doesn't document their behavior.
+    if (ARROW_PREDICT_FALSE(rhs < 0 || rhs >= std::numeric_limits<Arg0>::digits)) {
+      return lhs;
+    }
+    return lhs >> rhs;
+  }
+};
+
+struct ShiftRightChecked {
+  template <typename T, typename Arg0, typename Arg1>
+  static T Call(KernelContext*, Arg0 lhs, Arg1 rhs, Status* st) {
+    static_assert(std::is_same<T, Arg0>::value, "");
+    if (ARROW_PREDICT_FALSE(rhs < 0 || rhs >= std::numeric_limits<Arg0>::digits)) {
+      *st = Status::Invalid("shift amount must be >= 0 and less than precision of type");
+      return lhs;
+    }
+    return lhs >> rhs;
+  }
+};
+
 // Generate a kernel given an arithmetic functor
 template <template <typename... Args> class KernelGenerator, typename Op>
 ArrayKernelExec ArithmeticExecFromOp(detail::GetTypeId get_id) {
@@ -485,6 +585,54 @@ ArrayKernelExec ArithmeticExecFromOp(detail::GetTypeId get_id) {
   }
 }
 
+// Generate a kernel given a bitwise arithmetic functor. Assumes the
+// functor treats all integer types of equal width identically
+template <template <typename... Args> class KernelGenerator, typename Op>
+ArrayKernelExec TypeAgnosticBitWiseExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT8:
+    case Type::UINT8:
+      return KernelGenerator<UInt8Type, UInt8Type, Op>::Exec;
+    case Type::INT16:
+    case Type::UINT16:
+      return KernelGenerator<UInt16Type, UInt16Type, Op>::Exec;
+    case Type::INT32:
+    case Type::UINT32:
+      return KernelGenerator<UInt32Type, UInt32Type, Op>::Exec;
+    case Type::INT64:
+    case Type::UINT64:
+      return KernelGenerator<UInt64Type, UInt64Type, Op>::Exec;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
+template <template <typename... Args> class KernelGenerator, typename Op>
+ArrayKernelExec ShiftExecFromOp(detail::GetTypeId get_id) {
+  switch (get_id.id) {
+    case Type::INT8:
+      return KernelGenerator<Int8Type, Int8Type, Op>::Exec;
+    case Type::UINT8:
+      return KernelGenerator<UInt8Type, UInt8Type, Op>::Exec;
+    case Type::INT16:
+      return KernelGenerator<Int16Type, Int16Type, Op>::Exec;
+    case Type::UINT16:
+      return KernelGenerator<UInt16Type, UInt16Type, Op>::Exec;
+    case Type::INT32:
+      return KernelGenerator<Int32Type, Int32Type, Op>::Exec;
+    case Type::UINT32:
+      return KernelGenerator<UInt32Type, UInt32Type, Op>::Exec;
+    case Type::INT64:
+      return KernelGenerator<Int64Type, Int64Type, Op>::Exec;
+    case Type::UINT64:
+      return KernelGenerator<UInt64Type, UInt64Type, Op>::Exec;
+    default:
+      DCHECK(false);
+      return ExecFail;
+  }
+}
+
 Status CastBinaryDecimalArgs(const std::string& func_name,
                              std::vector<ValueDescr>* values) {
   auto& left_type = (*values)[0].type;
@@ -734,6 +882,28 @@ std::shared_ptr<ScalarFunction> MakeUnarySignedArithmeticFunctionNotNull(
   return func;
 }
 
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBitWiseFunctionNotNull(std::string name,
+                                                           const FunctionDoc* doc) {
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Binary(), doc);
+  for (const auto& ty : IntTypes()) {
+    auto exec = TypeAgnosticBitWiseExecFromOp<ScalarBinaryNotNullEqualTypes, Op>(ty);
+    DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
+  }
+  return func;
+}
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeShiftFunctionNotNull(std::string name,
+                                                         const FunctionDoc* doc) {
+  auto func = std::make_shared<ArithmeticFunction>(name, Arity::Binary(), doc);
+  for (const auto& ty : IntTypes()) {
+    auto exec = ShiftExecFromOp<ScalarBinaryNotNullEqualTypes, Op>(ty);
+    DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
+  }
+  return func;
+}
+
 const FunctionDoc absolute_value_doc{
     "Calculate the absolute value of the argument element-wise",
     ("Results will wrap around on integer overflow.\n"
@@ -820,6 +990,57 @@ const FunctionDoc pow_checked_doc{
     ("An error is returned when integer to negative integer power is encountered,\n"
      "or integer overflow is encountered."),
     {"base", "exponent"}};
+
+const FunctionDoc bit_wise_not_doc{
+    "Bit-wise negate the arguments element-wise", ("Null values return null."), {"x"}};
+
+const FunctionDoc bit_wise_and_doc{
+    "Bit-wise AND the arguments element-wise", ("Null values return null."), {"x", "y"}};
+
+const FunctionDoc bit_wise_or_doc{
+    "Bit-wise OR the arguments element-wise", ("Null values return null."), {"x", "y"}};
+
+const FunctionDoc bit_wise_xor_doc{
+    "Bit-wise XOR the arguments element-wise", ("Null values return null."), {"x", "y"}};
+
+const FunctionDoc shift_left_doc{
+    "Left shift `x` by `y`",
+    ("This function will return `x` if `y` (the amount to shift by) is: "
+     "(1) negative or (2) greater than or equal to the precision of `x`.\n"
+     "The shift operates as if on the two's complement representation of the number. "
+     "In other words, this is equivalent to multiplying `x` by 2 to the power `y`, "
+     "even if overflow occurs.\n"
+     "Use function \"shift_left_checked\" if you want an invalid shift amount to "
+     "return an error."),
+    {"x", "y"}};
+
+const FunctionDoc shift_left_checked_doc{
+    "Left shift `x` by `y` with invalid shift check",
+    ("This function will raise an error if `y` (the amount to shift by) is: "
+     "(1) negative or (2) greater than or equal to the precision of `x`. "
+     "The shift operates as if on the two's complement representation of the number. "
+     "In other words, this is equivalent to multiplying `x` by 2 to the power `y`, "
+     "even if overflow occurs.\n"
+     "See \"shift_left\" for a variant that doesn't fail for an invalid shift amount."),
+    {"x", "y"}};
+
+const FunctionDoc shift_right_doc{
+    "Right shift `x` by `y`",
+    ("Perform a logical shift for unsigned `x` and an arithmetic shift for signed `x`.\n"
+     "This function will return `x` if `y` (the amount to shift by) is: "
+     "(1) negative or (2) greater than or equal to the precision of `x`.\n"
+     "Use function \"shift_right_checked\" if you want an invalid shift amount to return "
+     "an error."),
+    {"x", "y"}};
+
+const FunctionDoc shift_right_checked_doc{
+    "Right shift `x` by `y` with invalid shift check",
+    ("Perform a logical shift for unsigned `x` and an arithmetic shift for signed `x`.\n"
+     "This function will raise an error if `y` (the amount to shift by) is: "
+     "(1) negative or (2) greater than or equal to the precision of `x`.\n"
+     "See \"shift_right\" for a variant that doesn't fail for an invalid shift amount"),
+    {"x", "y"}};
+
 }  // namespace
 
 void RegisterScalarArithmetic(FunctionRegistry* registry) {
@@ -903,6 +1124,44 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   auto power_checked =
       MakeArithmeticFunctionNotNull<PowerChecked>("power_checked", &pow_checked_doc);
   DCHECK_OK(registry->AddFunction(std::move(power_checked)));
+
+  // ----------------------------------------------------------------------
+  {
+    auto bit_wise_not = std::make_shared<ArithmeticFunction>(
+        "bit_wise_not", Arity::Unary(), &bit_wise_not_doc);
+    for (const auto& ty : IntTypes()) {
+      auto exec = TypeAgnosticBitWiseExecFromOp<ScalarUnaryNotNull, BitWiseNot>(ty);
+      DCHECK_OK(bit_wise_not->AddKernel({ty}, ty, exec));
+    }
+    DCHECK_OK(registry->AddFunction(std::move(bit_wise_not)));
+  }
+
+  auto bit_wise_and =
+      MakeBitWiseFunctionNotNull<BitWiseAnd>("bit_wise_and", &bit_wise_and_doc);
+  DCHECK_OK(registry->AddFunction(std::move(bit_wise_and)));
+
+  auto bit_wise_or =
+      MakeBitWiseFunctionNotNull<BitWiseOr>("bit_wise_or", &bit_wise_or_doc);
+  DCHECK_OK(registry->AddFunction(std::move(bit_wise_or)));
+
+  auto bit_wise_xor =
+      MakeBitWiseFunctionNotNull<BitWiseXor>("bit_wise_xor", &bit_wise_xor_doc);
+  DCHECK_OK(registry->AddFunction(std::move(bit_wise_xor)));
+
+  auto shift_left = MakeShiftFunctionNotNull<ShiftLeft>("shift_left", &shift_left_doc);
+  DCHECK_OK(registry->AddFunction(std::move(shift_left)));
+
+  auto shift_left_checked = MakeShiftFunctionNotNull<ShiftLeftChecked>(
+      "shift_left_checked", &shift_left_checked_doc);
+  DCHECK_OK(registry->AddFunction(std::move(shift_left_checked)));
+
+  auto shift_right =
+      MakeShiftFunctionNotNull<ShiftRight>("shift_right", &shift_right_doc);
+  DCHECK_OK(registry->AddFunction(std::move(shift_right)));
+
+  auto shift_right_checked = MakeShiftFunctionNotNull<ShiftRightChecked>(
+      "shift_right_checked", &shift_right_checked_doc);
+  DCHECK_OK(registry->AddFunction(std::move(shift_right_checked)));
 }
 
 }  // namespace internal
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
index ae2f55c..a94eabb 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
@@ -299,6 +299,66 @@ class TestBinaryArithmeticUnsigned : public TestBinaryArithmeticIntegral<T> {};
 template <typename T>
 class TestBinaryArithmeticFloating : public TestBinaryArithmetic<T> {};
 
+template <typename T>
+class TestBitWiseArithmetic : public TestBase {
+ protected:
+  using ArrowType = T;
+  using CType = typename ArrowType::c_type;
+
+  static std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+
+  void AssertUnaryOp(const std::string& func, const std::vector<uint8_t>& args,
+                     const std::vector<uint8_t>& expected) {
+    auto input = ExpandByteArray(args);
+    auto output = ExpandByteArray(expected);
+    ASSERT_OK_AND_ASSIGN(Datum actual, CallFunction(func, {input}));
+    ValidateAndAssertEqual(actual.make_array(), output);
+    for (int64_t i = 0; i < output->length(); i++) {
+      ASSERT_OK_AND_ASSIGN(Datum actual, CallFunction(func, {*input->GetScalar(i)}));
+      const auto expected_scalar = *output->GetScalar(i);
+      AssertScalarsEqual(*expected_scalar, *actual.scalar(), /*verbose=*/true);
+    }
+  }
+
+  void AssertBinaryOp(const std::string& func, const std::vector<uint8_t>& arg0,
+                      const std::vector<uint8_t>& arg1,
+                      const std::vector<uint8_t>& expected) {
+    auto input0 = ExpandByteArray(arg0);
+    auto input1 = ExpandByteArray(arg1);
+    auto output = ExpandByteArray(expected);
+    ASSERT_OK_AND_ASSIGN(Datum actual, CallFunction(func, {input0, input1}));
+    ValidateAndAssertEqual(actual.make_array(), output);
+    for (int64_t i = 0; i < output->length(); i++) {
+      ASSERT_OK_AND_ASSIGN(Datum actual, CallFunction(func, {*input0->GetScalar(i),
+                                                             *input1->GetScalar(i)}));
+      const auto expected_scalar = *output->GetScalar(i);
+      AssertScalarsEqual(*expected_scalar, *actual.scalar(), /*verbose=*/true);
+    }
+  }
+
+  // To make it easier to test different widths, tests give bytes which get repeated to
+  // make an array of the actual type
+  std::shared_ptr<Array> ExpandByteArray(const std::vector<uint8_t>& values) {
+    std::vector<CType> c_values(values.size() + 1);
+    for (size_t i = 0; i < values.size(); i++) {
+      std::memset(&c_values[i], values[i], sizeof(CType));
+    }
+    std::vector<bool> valid(values.size() + 1, true);
+    valid.back() = false;
+    std::shared_ptr<Array> arr;
+    ArrayFromVector<ArrowType>(valid, c_values, &arr);
+    return arr;
+  }
+
+  void ValidateAndAssertEqual(const std::shared_ptr<Array>& actual,
+                              const std::shared_ptr<Array>& expected) {
+    ASSERT_OK(actual->ValidateFull());
+    AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+  }
+};
+
 // InputType - OutputType pairs
 using IntegralTypes = testing::Types<Int8Type, Int16Type, Int32Type, Int64Type, UInt8Type,
                                      UInt16Type, UInt32Type, UInt64Type>;
@@ -321,6 +381,31 @@ TYPED_TEST_SUITE(TestBinaryArithmeticSigned, SignedIntegerTypes);
 TYPED_TEST_SUITE(TestBinaryArithmeticUnsigned, UnsignedIntegerTypes);
 TYPED_TEST_SUITE(TestBinaryArithmeticFloating, FloatingTypes);
 
+TYPED_TEST_SUITE(TestBitWiseArithmetic, IntegralTypes);
+
+TYPED_TEST(TestBitWiseArithmetic, BitWiseNot) {
+  this->AssertUnaryOp("bit_wise_not", std::vector<uint8_t>{0x00, 0x55, 0xAA, 0xFF},
+                      std::vector<uint8_t>{0xFF, 0xAA, 0x55, 0x00});
+}
+
+TYPED_TEST(TestBitWiseArithmetic, BitWiseAnd) {
+  this->AssertBinaryOp("bit_wise_and", std::vector<uint8_t>{0x00, 0xFF, 0x00, 0xFF},
+                       std::vector<uint8_t>{0x00, 0x00, 0xFF, 0xFF},
+                       std::vector<uint8_t>{0x00, 0x00, 0x00, 0xFF});
+}
+
+TYPED_TEST(TestBitWiseArithmetic, BitWiseOr) {
+  this->AssertBinaryOp("bit_wise_or", std::vector<uint8_t>{0x00, 0xFF, 0x00, 0xFF},
+                       std::vector<uint8_t>{0x00, 0x00, 0xFF, 0xFF},
+                       std::vector<uint8_t>{0x00, 0xFF, 0xFF, 0xFF});
+}
+
+TYPED_TEST(TestBitWiseArithmetic, BitWiseXor) {
+  this->AssertBinaryOp("bit_wise_xor", std::vector<uint8_t>{0x00, 0xFF, 0x00, 0xFF},
+                       std::vector<uint8_t>{0x00, 0x00, 0xFF, 0xFF},
+                       std::vector<uint8_t>{0x00, 0xFF, 0xFF, 0x00});
+}
+
 TYPED_TEST(TestBinaryArithmeticIntegral, Add) {
   for (auto check_overflow : {false, true}) {
     this->SetOverflowCheck(check_overflow);
@@ -1511,5 +1596,124 @@ TEST(TestBinaryArithmeticDecimal, Divide) {
   }
 }
 
+TYPED_TEST(TestBinaryArithmeticIntegral, ShiftLeft) {
+  for (auto check_overflow : {false, true}) {
+    this->SetOverflowCheck(check_overflow);
+
+    this->AssertBinop(ShiftLeft, "[]", "[]", "[]");
+    this->AssertBinop(ShiftLeft, "[0, 1, 2, 3]", "[2, 3, 4, 5]", "[0, 8, 32, 96]");
+    // Nulls on one side
+    this->AssertBinop(ShiftLeft, "[0, null, 2, 3]", "[2, 3, 4, 5]", "[0, null, 32, 96]");
+    this->AssertBinop(ShiftLeft, "[0, 1, 2, 3]", "[2, 3, null, 5]", "[0, 8, null, 96]");
+    // Nulls on both sides
+    this->AssertBinop(ShiftLeft, "[0, null, 2, 3]", "[2, 3, null, 5]",
+                      "[0, null, null, 96]");
+    // All nulls
+    this->AssertBinop(ShiftLeft, "[null]", "[null]", "[null]");
+
+    // Scalar on the left
+    this->AssertBinop(ShiftLeft, 2, "[null, 5]", "[null, 64]");
+    this->AssertBinop(ShiftLeft, this->MakeNullScalar(), "[null, 5]", "[null, null]");
+    // Scalar on the right
+    this->AssertBinop(ShiftLeft, "[null, 5]", 3, "[null, 40]");
+    this->AssertBinop(ShiftLeft, "[null, 5]", this->MakeNullScalar(), "[null, null]");
+  }
+}
+
+TYPED_TEST(TestBinaryArithmeticIntegral, ShiftRight) {
+  for (auto check_overflow : {false, true}) {
+    this->SetOverflowCheck(check_overflow);
+
+    this->AssertBinop(ShiftRight, "[]", "[]", "[]");
+    this->AssertBinop(ShiftRight, "[0, 1, 4, 8]", "[1, 1, 1, 4]", "[0, 0, 2, 0]");
+    // Nulls on one side
+    this->AssertBinop(ShiftRight, "[0, null, 4, 8]", "[1, 1, 1, 4]", "[0, null, 2, 0]");
+    this->AssertBinop(ShiftRight, "[0, 1, 4, 8]", "[1, 1, null, 4]", "[0, 0, null, 0]");
+    // Nulls on both sides
+    this->AssertBinop(ShiftRight, "[0, null, 4, 8]", "[1, 1, null, 4]",
+                      "[0, null, null, 0]");
+    // All nulls
+    this->AssertBinop(ShiftRight, "[null]", "[null]", "[null]");
+
+    // Scalar on the left
+    this->AssertBinop(ShiftRight, 64, "[null, 2, 6]", "[null, 16, 1]");
+    this->AssertBinop(ShiftRight, this->MakeNullScalar(), "[null, 2, 6]",
+                      "[null, null, null]");
+    // Scalar on the right
+    this->AssertBinop(ShiftRight, "[null, 3, 96]", 3, "[null, 0, 12]");
+    this->AssertBinop(ShiftRight, "[null, 3, 96]", this->MakeNullScalar(),
+                      "[null, null, null]");
+  }
+}
+
+TYPED_TEST(TestBinaryArithmeticSigned, ShiftLeftOverflowRaises) {
+  using CType = typename TestFixture::CType;
+  const CType bit_width = static_cast<CType>(std::numeric_limits<CType>::digits);
+  const CType min = std::numeric_limits<CType>::min();
+  this->SetOverflowCheck(true);
+
+  this->AssertBinop(ShiftLeft, "[1]", MakeArray(bit_width - 1),
+                    MakeArray(static_cast<CType>(1) << (bit_width - 1)));
+  this->AssertBinop(ShiftLeft, "[2]", MakeArray(bit_width - 2),
+                    MakeArray(static_cast<CType>(1) << (bit_width - 1)));
+  // Shift a bit into the sign bit
+  this->AssertBinop(ShiftLeft, "[2]", MakeArray(bit_width - 1), MakeArray(min));
+  // Shift a bit past the sign bit
+  this->AssertBinop(ShiftLeft, "[4]", MakeArray(bit_width - 1), "[0]");
+  this->AssertBinop(ShiftLeft, MakeArray(min), "[1]", "[0]");
+  this->AssertBinopRaises(ShiftLeft, "[1, 2]", "[1, -1]",
+                          "shift amount must be >= 0 and less than precision of type");
+  this->AssertBinopRaises(ShiftLeft, "[1]", MakeArray(bit_width),
+                          "shift amount must be >= 0 and less than precision of type");
+
+  this->SetOverflowCheck(false);
+  this->AssertBinop(ShiftLeft, "[1, 1]", MakeArray(-1, bit_width), "[1, 1]");
+}
+
+TYPED_TEST(TestBinaryArithmeticSigned, ShiftRightOverflowRaises) {
+  using CType = typename TestFixture::CType;
+  const CType bit_width = static_cast<CType>(std::numeric_limits<CType>::digits);
+  const CType max = std::numeric_limits<CType>::max();
+  const CType min = std::numeric_limits<CType>::min();
+  this->SetOverflowCheck(true);
+
+  this->AssertBinop(ShiftRight, MakeArray(max), MakeArray(bit_width - 1), "[1]");
+  this->AssertBinop(ShiftRight, "[-1, -1]", "[1, 5]", "[-1, -1]");
+  this->AssertBinop(ShiftRight, MakeArray(min), "[1]", MakeArray(min / 2));
+  this->AssertBinopRaises(ShiftRight, "[1, 2]", "[1, -1]",
+                          "shift amount must be >= 0 and less than precision of type");
+  this->AssertBinopRaises(ShiftRight, "[1]", MakeArray(bit_width),
+                          "shift amount must be >= 0 and less than precision of type");
+
+  this->SetOverflowCheck(false);
+  this->AssertBinop(ShiftRight, "[1, 1]", MakeArray(-1, bit_width), "[1, 1]");
+}
+
+TYPED_TEST(TestBinaryArithmeticUnsigned, ShiftLeftOverflowRaises) {
+  using CType = typename TestFixture::CType;
+  const CType bit_width = static_cast<CType>(std::numeric_limits<CType>::digits);
+  this->SetOverflowCheck(true);
+
+  this->AssertBinop(ShiftLeft, "[1]", MakeArray(bit_width - 1),
+                    MakeArray(static_cast<CType>(1) << (bit_width - 1)));
+  this->AssertBinop(ShiftLeft, "[2]", MakeArray(bit_width - 2),
+                    MakeArray(static_cast<CType>(1) << (bit_width - 1)));
+  this->AssertBinop(ShiftLeft, "[2]", MakeArray(bit_width - 1), "[0]");
+  this->AssertBinop(ShiftLeft, "[4]", MakeArray(bit_width - 1), "[0]");
+  this->AssertBinopRaises(ShiftLeft, "[1]", MakeArray(bit_width),
+                          "shift amount must be >= 0 and less than precision of type");
+}
+
+TYPED_TEST(TestBinaryArithmeticUnsigned, ShiftRightOverflowRaises) {
+  using CType = typename TestFixture::CType;
+  const CType bit_width = static_cast<CType>(std::numeric_limits<CType>::digits);
+  const CType max = std::numeric_limits<CType>::max();
+  this->SetOverflowCheck(true);
+
+  this->AssertBinop(ShiftRight, MakeArray(max), MakeArray(bit_width - 1), "[1]");
+  this->AssertBinopRaises(ShiftRight, "[1]", MakeArray(bit_width),
+                          "shift amount must be >= 0 and less than precision of type");
+}
+
 }  // namespace compute
 }  // namespace arrow
diff --git a/docs/source/cpp/compute.rst b/docs/source/cpp/compute.rst
index a4ca8f9..c4ca4d3 100644
--- a/docs/source/cpp/compute.rst
+++ b/docs/source/cpp/compute.rst
@@ -293,24 +293,51 @@ an ``Invalid`` :class:`Status` when overflow is detected.
 
 * \(1) Precision and scale of computed DECIMAL results
 
-+------------+---------------------------------------------+
-| Operation  | Result precision and scale                  |
-+============+=============================================+
-| | add      | | scale = max(s1, s2)                       |
-| | subtract | | precision = max(p1-s1, p2-s2) + 1 + scale |
-+------------+---------------------------------------------+
-| multiply   | | scale = s1 + s2                           |
-|            | | precision = p1 + p2 + 1                   |
-+------------+---------------------------------------------+
-| divide     | | scale = max(4, s1 + p2 - s2 + 1)          |
-|            | | precision = p1 - s1 + s2 + scale          |
-+------------+---------------------------------------------+
-
-It's compatible with Redshift's decimal promotion rules. All decimal digits
-are preserved for `add`, `subtract` and `multiply` operations. The result
-precision of `divide` is at least the sum of precisions of both operands with
-enough scale kept. Error is returned if the result precision is beyond the
-decimal value range.
+  +------------+---------------------------------------------+
+  | Operation  | Result precision and scale                  |
+  +============+=============================================+
+  | | add      | | scale = max(s1, s2)                       |
+  | | subtract | | precision = max(p1-s1, p2-s2) + 1 + scale |
+  +------------+---------------------------------------------+
+  | multiply   | | scale = s1 + s2                           |
+  |            | | precision = p1 + p2 + 1                   |
+  +------------+---------------------------------------------+
+  | divide     | | scale = max(4, s1 + p2 - s2 + 1)          |
+  |            | | precision = p1 - s1 + s2 + scale          |
+  +------------+---------------------------------------------+
+
+  It's compatible with Redshift's decimal promotion rules. All decimal digits
+  are preserved for `add`, `subtract` and `multiply` operations. The result
+  precision of `divide` is at least the sum of precisions of both operands with
+  enough scale kept. Error is returned if the result precision is beyond the
+  decimal value range.
+
+Bit-wise functions
+~~~~~~~~~~~~~~~~~~
+
++--------------------------+------------+--------------------+---------------------+
+| Function name            | Arity      | Input types        | Output type         |
++==========================+============+====================+=====================+
+| bit_wise_and             | Binary     | Numeric            | Numeric             |
++--------------------------+------------+--------------------+---------------------+
+| bit_wise_not             | Unary      | Numeric            | Numeric             |
++--------------------------+------------+--------------------+---------------------+
+| bit_wise_or              | Binary     | Numeric            | Numeric             |
++--------------------------+------------+--------------------+---------------------+
+| bit_wise_xor             | Binary     | Numeric            | Numeric             |
++--------------------------+------------+--------------------+---------------------+
+| shift_left               | Binary     | Numeric            | Numeric             |
++--------------------------+------------+--------------------+---------------------+
+| shift_left_checked       | Binary     | Numeric            | Numeric (1)         |
++--------------------------+------------+--------------------+---------------------+
+| shift_right              | Binary     | Numeric            | Numeric             |
++--------------------------+------------+--------------------+---------------------+
+| shift_right_checked      | Binary     | Numeric            | Numeric (1)         |
++--------------------------+------------+--------------------+---------------------+
+
+* \(1) An error is emitted if the shift amount (i.e. the second input) is
+  out of bounds for the data type.  However, an overflow when shifting the
+  first input is not error (truncated bits are silently discarded).
 
 Comparisons
 ~~~~~~~~~~~
diff --git a/docs/source/python/api/compute.rst b/docs/source/python/api/compute.rst
index 34626e2..461803d 100644
--- a/docs/source/python/api/compute.rst
+++ b/docs/source/python/api/compute.rst
@@ -58,6 +58,20 @@ throws an ``ArrowInvalid`` exception when overflow is detected.
    subtract_checked
    power
    power_checked
+   shift_left
+   shift_left_checked
+   shift_right
+   shift_right_checked
+
+Bit-wise operations do not offer (or need) a checked variant.
+
+.. autosummary::
+   :toctree: ../generated/
+
+   bit_wise_and
+   bit_wise_not
+   bit_wise_or
+   bit_wise_xor
 
 Comparisons
 -----------