You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/11/10 16:37:56 UTC

[GitHub] [arrow] pitrou commented on a diff in pull request #14550: ARROW-17301: [C++] Implement compute function "binary_slice"

pitrou commented on code in PR #14550:
URL: https://github.com/apache/arrow/pull/14550#discussion_r1019355193


##########
cpp/src/arrow/compute/kernels/scalar_string_ascii.cc:
##########
@@ -2409,6 +2409,172 @@ void AddAsciiStringReplaceSlice(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(func)));
 }
 
+// ----------------------------------------------------------------------
+// Slice
+
+namespace {
+struct SliceBytesTransform : StringSliceTransformBase {
+  int64_t MaxCodeunits(int64_t ninputs, int64_t input_bytes) override {
+    const SliceOptions& opt = *this->options;
+    if ((opt.start >= 0) != (opt.stop >= 0)) {
+      // If start and stop don't have the same sign, we can't guess an upper bound
+      // on the resulting slice lengths, so return a worst case estimate.
+      return input_bytes;
+    }
+    int64_t max_slice_bytes = (opt.stop - opt.start + opt.step - 1) / opt.step;
+    return std::min(input_bytes, ninputs * std::max<int64_t>(0, max_slice_bytes));
+  }
+
+  int64_t Transform(const uint8_t* input, int64_t input_string_bytes, uint8_t* output) {
+    if (options->step >= 1) {
+      return SliceForward(input, input_string_bytes, output);
+    }
+    return SliceBackward(input, input_string_bytes, output);
+  }
+
+  int64_t SliceForward(const uint8_t* input, int64_t input_string_bytes,
+                       uint8_t* output) {
+    // Slice in forward order (step > 0)
+    const SliceOptions& opt = *this->options;
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_bytes;
+    const uint8_t* begin_sliced;
+    const uint8_t* end_sliced;
+
+    if (!input_string_bytes) {
+      return 0;
+    }
+    // First, compute begin_sliced and end_sliced
+    if (opt.start >= 0) {
+      // start counting from the left
+      begin_sliced = std::min(begin + opt.start, end);
+      if (opt.stop > opt.start) {
+        // continue counting from begin_sliced
+        const int64_t length = opt.stop - opt.start;
+        end_sliced = std::min(begin_sliced + length, end);
+      } else if (opt.stop < 0) {
+        // from the end
+        end_sliced = std::max(end + opt.stop, begin_sliced);
+      } else {
+        // zero length slice
+        return 0;
+      }
+    } else {
+      // start counting from the right
+      begin_sliced = std::max(end + opt.start, begin);
+      if (opt.stop > 0) {
+        // continue counting from the left, we cannot start from begin_sliced because we
+        // don't know how many bytes are between begin and begin_sliced
+        end_sliced = std::min(begin + opt.stop, end);
+        // and therefore we also needs this
+        if (end_sliced <= begin_sliced) {
+          // zero length slice
+          return 0;
+        }
+      } else if ((opt.stop < 0) && (opt.stop > opt.start)) {
+        // stop is negative, but larger than start, so we count again from the right
+        // in some cases we can optimize this, depending on the shortest path (from end
+        // or begin_sliced), but begin_sliced and opt.start can be 'out of sync',
+        // for instance when start=-100, when the string length is only 10.
+        end_sliced = std::max(end + opt.stop, begin_sliced);
+      } else {
+        // zero length slice
+        return 0;
+      }
+    }
+
+    // Second, copy computed slice to output
+    DCHECK(begin_sliced <= end_sliced);
+    if (opt.step == 1) {
+      // fast case, where we simply can finish with a memcpy
+      std::copy(begin_sliced, end_sliced, output);
+      return end_sliced - begin_sliced;
+    }
+
+    uint8_t* dest = output;
+    const uint8_t* i = begin_sliced;
+
+    while (i < end_sliced) {
+      *dest = *i;
+      // skip step codeunits
+      i += opt.step;
+      dest++;
+    }
+    return dest - output;
+  }
+
+  int64_t SliceBackward(const uint8_t* input, int64_t input_string_bytes,
+                        uint8_t* output) {
+    // Slice in reverse order (step < 0)
+    const SliceOptions& opt = *this->options;
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_bytes;
+    const uint8_t* begin_sliced = begin;
+    const uint8_t* end_sliced = end;
+
+    if (!input_string_bytes) {
+      return 0;
+    }
+
+    if (opt.start >= 0) {
+      // +1 because begin_sliced acts as as the end of a reverse iterator
+      begin_sliced = std::min(begin + opt.start + 1, end);
+    } else {
+      // -1 because start=-1 means the last byte, which is 0 advances
+      begin_sliced = std::max(end + opt.start + 1, begin);
+    }
+    begin_sliced--;
+
+    // similar to opt.start
+    if (opt.stop >= 0) {
+      end_sliced = std::min(begin + opt.stop + 1, end);
+    } else {
+      end_sliced = std::max(end + opt.stop + 1, begin);
+    }
+    end_sliced--;
+
+    // Copy computed slice to output
+    uint8_t* dest = output;
+    const uint8_t* i = begin_sliced;
+    while (i > end_sliced) {
+      // write a single codepoint
+      *dest = *i;
+      // and skip the remainder
+      i += opt.step;
+      dest++;
+    }
+
+    return dest - output;
+  }
+};
+
+template <typename Type>
+using SliceBytes = StringTransformExec<Type, SliceBytesTransform>;
+
+}  // namespace
+
+const FunctionDoc binary_slice_bytes_doc(
+    "Slice string",

Review Comment:
   ```suggestion
       "Slice binary string",
   ```



##########
cpp/src/arrow/compute/kernels/scalar_string_ascii.cc:
##########
@@ -2409,6 +2409,172 @@ void AddAsciiStringReplaceSlice(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(func)));
 }
 
+// ----------------------------------------------------------------------
+// Slice
+
+namespace {
+struct SliceBytesTransform : StringSliceTransformBase {
+  int64_t MaxCodeunits(int64_t ninputs, int64_t input_bytes) override {
+    const SliceOptions& opt = *this->options;
+    if ((opt.start >= 0) != (opt.stop >= 0)) {
+      // If start and stop don't have the same sign, we can't guess an upper bound
+      // on the resulting slice lengths, so return a worst case estimate.
+      return input_bytes;
+    }
+    int64_t max_slice_bytes = (opt.stop - opt.start + opt.step - 1) / opt.step;
+    return std::min(input_bytes, ninputs * std::max<int64_t>(0, max_slice_bytes));
+  }
+
+  int64_t Transform(const uint8_t* input, int64_t input_string_bytes, uint8_t* output) {
+    if (options->step >= 1) {
+      return SliceForward(input, input_string_bytes, output);
+    }
+    return SliceBackward(input, input_string_bytes, output);
+  }
+
+  int64_t SliceForward(const uint8_t* input, int64_t input_string_bytes,
+                       uint8_t* output) {
+    // Slice in forward order (step > 0)
+    const SliceOptions& opt = *this->options;
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_bytes;
+    const uint8_t* begin_sliced;
+    const uint8_t* end_sliced;
+
+    if (!input_string_bytes) {
+      return 0;
+    }
+    // First, compute begin_sliced and end_sliced
+    if (opt.start >= 0) {
+      // start counting from the left
+      begin_sliced = std::min(begin + opt.start, end);
+      if (opt.stop > opt.start) {
+        // continue counting from begin_sliced
+        const int64_t length = opt.stop - opt.start;
+        end_sliced = std::min(begin_sliced + length, end);
+      } else if (opt.stop < 0) {
+        // from the end
+        end_sliced = std::max(end + opt.stop, begin_sliced);
+      } else {
+        // zero length slice
+        return 0;
+      }
+    } else {
+      // start counting from the right
+      begin_sliced = std::max(end + opt.start, begin);
+      if (opt.stop > 0) {
+        // continue counting from the left, we cannot start from begin_sliced because we
+        // don't know how many bytes are between begin and begin_sliced
+        end_sliced = std::min(begin + opt.stop, end);
+        // and therefore we also needs this
+        if (end_sliced <= begin_sliced) {
+          // zero length slice
+          return 0;
+        }
+      } else if ((opt.stop < 0) && (opt.stop > opt.start)) {
+        // stop is negative, but larger than start, so we count again from the right
+        // in some cases we can optimize this, depending on the shortest path (from end
+        // or begin_sliced), but begin_sliced and opt.start can be 'out of sync',
+        // for instance when start=-100, when the string length is only 10.
+        end_sliced = std::max(end + opt.stop, begin_sliced);
+      } else {
+        // zero length slice
+        return 0;
+      }
+    }
+
+    // Second, copy computed slice to output
+    DCHECK(begin_sliced <= end_sliced);
+    if (opt.step == 1) {
+      // fast case, where we simply can finish with a memcpy
+      std::copy(begin_sliced, end_sliced, output);
+      return end_sliced - begin_sliced;
+    }
+
+    uint8_t* dest = output;
+    const uint8_t* i = begin_sliced;
+
+    while (i < end_sliced) {
+      *dest = *i;
+      // skip step codeunits
+      i += opt.step;
+      dest++;
+    }
+    return dest - output;
+  }
+
+  int64_t SliceBackward(const uint8_t* input, int64_t input_string_bytes,
+                        uint8_t* output) {
+    // Slice in reverse order (step < 0)
+    const SliceOptions& opt = *this->options;
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_bytes;
+    const uint8_t* begin_sliced = begin;
+    const uint8_t* end_sliced = end;
+
+    if (!input_string_bytes) {
+      return 0;
+    }
+
+    if (opt.start >= 0) {
+      // +1 because begin_sliced acts as as the end of a reverse iterator
+      begin_sliced = std::min(begin + opt.start + 1, end);
+    } else {
+      // -1 because start=-1 means the last byte, which is 0 advances
+      begin_sliced = std::max(end + opt.start + 1, begin);
+    }
+    begin_sliced--;
+
+    // similar to opt.start
+    if (opt.stop >= 0) {
+      end_sliced = std::min(begin + opt.stop + 1, end);
+    } else {
+      end_sliced = std::max(end + opt.stop + 1, begin);
+    }
+    end_sliced--;
+
+    // Copy computed slice to output
+    uint8_t* dest = output;
+    const uint8_t* i = begin_sliced;
+    while (i > end_sliced) {
+      // write a single codepoint
+      *dest = *i;
+      // and skip the remainder
+      i += opt.step;
+      dest++;
+    }
+
+    return dest - output;
+  }
+};
+
+template <typename Type>
+using SliceBytes = StringTransformExec<Type, SliceBytesTransform>;
+
+}  // namespace
+
+const FunctionDoc binary_slice_bytes_doc(
+    "Slice string",
+    ("For each string in `strings`, emit the substring defined by\n"
+     "(`start`, `stop`, `step`) as given by `SliceOptions` where `start` is\n"
+     "inclusive and `stop` is exclusive. All three values are measured in\n"
+     "bytes.\n"
+     "If `step` is negative, the string will be advanced in reversed order.\n"
+     "An error is raised if `step` is zero.\n"
+     "Null inputs emit null."),
+    {"strings"}, "SliceOptions", /*options_required=*/true);
+
+void AddAsciiStringSlice(FunctionRegistry* registry) {
+  auto func = std::make_shared<ScalarFunction>("binary_slice_bytes", Arity::Unary(),

Review Comment:
   I think `binary_slice` is explanatory enough.



##########
cpp/src/arrow/compute/kernels/scalar_string_test.cc:
##########
@@ -2119,6 +2119,96 @@ TYPED_TEST(TestStringKernels, SliceCodeunitsNegPos) {
 
 #endif  // ARROW_WITH_UTF8PROC
 
+TYPED_TEST(TestBaseBinaryKernels, SliceBytesBasic) {
+  SliceOptions options{2, 4};
+  this->CheckUnary("binary_slice_bytes", R"(["foo", "fo", null, "foo "])", this->type(),
+                   R"(["o", "", null, "o "])", &options);
+
+  // end is beyond 0, but before start (hence empty)
+  SliceOptions options_edgecase_1{-3, 1};
+  this->CheckUnary("binary_slice_bytes", R"(["foods"])", this->type(), R"([""])",
+                   &options_edgecase_1);
+
+  // this is a safeguard agains an optimization path possible, but actually a tricky case
+  SliceOptions options_edgecase_2{-6, -2};
+  this->CheckUnary("binary_slice_bytes", R"(["foods"])", this->type(), R"(["foo"])",
+                   &options_edgecase_2);
+
+  auto input = ArrayFromJSON(this->type(), R"(["foods"])");
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      testing::HasSubstr(
+          "Function 'binary_slice_bytes' cannot be called without options"),
+      CallFunction("binary_slice_bytes", {input}));
+
+  SliceOptions options_invalid{2, 4, 0};
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid, testing::HasSubstr("Slice step cannot be zero"),
+      CallFunction("binary_slice_bytes", {input}, &options_invalid));
+}
+
+TYPED_TEST(TestBaseBinaryKernels, SliceBytesPosPos) {
+  SliceOptions options{2, 4};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",
+                   this->type(), R"(["", "", "", "o", "od", "od"])", &options);
+  SliceOptions options_step{1, 5, 2};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",
+                   this->type(), R"(["", "", "o", "o", "od", "od"])", &options_step);
+  SliceOptions options_step_neg{5, 1, -2};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",
+                   this->type(), R"(["", "", "", "o", "d", "so"])", &options_step_neg);
+  options_step_neg.stop = 0;
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food","foods"])",
+                   this->type(), R"(["", "", "o", "o", "do", "so"])", &options_step_neg);
+}
+
+TYPED_TEST(TestBaseBinaryKernels, SliceBytesPosNeg) {
+  SliceOptions options{2, -1};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",

Review Comment:
   These tests would be better without any letter repetition in the source values. Otherwise the results might end up correct even with an incorrect implementation.



##########
r/src/compute.cpp:
##########
@@ -449,7 +449,7 @@ std::shared_ptr<arrow::compute::FunctionOptions> make_compute_options(
     return std::make_shared<Options>(cpp11::as_cpp<std::string>(options["characters"]));
   }
 
-  if (func_name == "utf8_slice_codeunits") {
+  if (func_name == "utf8_slice_codeunits" || func_name == "binary_slice_bytes") {

Review Comment:
   Should a test be created on the R side?
   @thisisnic @paleolimbot Perhaps one of you can help.



##########
python/pyarrow/tests/test_compute.py:
##########
@@ -536,6 +537,24 @@ def test_slice_compatibility():
                                                start, stop, step) == result
 
 
+def test_binary_slice_compatibility():
+    arr = pa.array((el.encode('ascii')
+                   for el in ["", "a", "ab", "abc", "abcd", "abcde"]))

Review Comment:
   Can write this in more idiomatic way. Also, it's nicer with some non-ASCII data:
   ```suggestion
       arr = pa.array([b"", b"a", b"a\xff", b"a\xffc", b"a\xffcd", b"a\xffcde"])
   ```
   



##########
cpp/src/arrow/compute/kernels/scalar_string_ascii.cc:
##########
@@ -2409,6 +2409,172 @@ void AddAsciiStringReplaceSlice(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(func)));
 }
 
+// ----------------------------------------------------------------------
+// Slice
+
+namespace {
+struct SliceBytesTransform : StringSliceTransformBase {
+  int64_t MaxCodeunits(int64_t ninputs, int64_t input_bytes) override {
+    const SliceOptions& opt = *this->options;
+    if ((opt.start >= 0) != (opt.stop >= 0)) {
+      // If start and stop don't have the same sign, we can't guess an upper bound
+      // on the resulting slice lengths, so return a worst case estimate.
+      return input_bytes;
+    }
+    int64_t max_slice_bytes = (opt.stop - opt.start + opt.step - 1) / opt.step;
+    return std::min(input_bytes, ninputs * std::max<int64_t>(0, max_slice_bytes));
+  }
+
+  int64_t Transform(const uint8_t* input, int64_t input_string_bytes, uint8_t* output) {
+    if (options->step >= 1) {
+      return SliceForward(input, input_string_bytes, output);
+    }
+    return SliceBackward(input, input_string_bytes, output);
+  }
+
+  int64_t SliceForward(const uint8_t* input, int64_t input_string_bytes,
+                       uint8_t* output) {
+    // Slice in forward order (step > 0)
+    const SliceOptions& opt = *this->options;
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_bytes;
+    const uint8_t* begin_sliced;
+    const uint8_t* end_sliced;
+
+    if (!input_string_bytes) {
+      return 0;
+    }
+    // First, compute begin_sliced and end_sliced
+    if (opt.start >= 0) {
+      // start counting from the left
+      begin_sliced = std::min(begin + opt.start, end);
+      if (opt.stop > opt.start) {
+        // continue counting from begin_sliced
+        const int64_t length = opt.stop - opt.start;
+        end_sliced = std::min(begin_sliced + length, end);
+      } else if (opt.stop < 0) {
+        // from the end
+        end_sliced = std::max(end + opt.stop, begin_sliced);
+      } else {
+        // zero length slice
+        return 0;
+      }
+    } else {
+      // start counting from the right
+      begin_sliced = std::max(end + opt.start, begin);
+      if (opt.stop > 0) {
+        // continue counting from the left, we cannot start from begin_sliced because we
+        // don't know how many bytes are between begin and begin_sliced
+        end_sliced = std::min(begin + opt.stop, end);
+        // and therefore we also needs this

Review Comment:
   ```suggestion
           // and therefore we also need this
   ```



##########
cpp/src/arrow/compute/kernels/scalar_string_test.cc:
##########
@@ -2119,6 +2119,96 @@ TYPED_TEST(TestStringKernels, SliceCodeunitsNegPos) {
 
 #endif  // ARROW_WITH_UTF8PROC
 
+TYPED_TEST(TestBaseBinaryKernels, SliceBytesBasic) {
+  SliceOptions options{2, 4};
+  this->CheckUnary("binary_slice_bytes", R"(["foo", "fo", null, "foo "])", this->type(),
+                   R"(["o", "", null, "o "])", &options);
+
+  // end is beyond 0, but before start (hence empty)
+  SliceOptions options_edgecase_1{-3, 1};
+  this->CheckUnary("binary_slice_bytes", R"(["foods"])", this->type(), R"([""])",
+                   &options_edgecase_1);
+
+  // this is a safeguard agains an optimization path possible, but actually a tricky case
+  SliceOptions options_edgecase_2{-6, -2};
+  this->CheckUnary("binary_slice_bytes", R"(["foods"])", this->type(), R"(["foo"])",
+                   &options_edgecase_2);
+
+  auto input = ArrayFromJSON(this->type(), R"(["foods"])");
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      testing::HasSubstr(
+          "Function 'binary_slice_bytes' cannot be called without options"),
+      CallFunction("binary_slice_bytes", {input}));
+
+  SliceOptions options_invalid{2, 4, 0};
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid, testing::HasSubstr("Slice step cannot be zero"),
+      CallFunction("binary_slice_bytes", {input}, &options_invalid));
+}
+
+TYPED_TEST(TestBaseBinaryKernels, SliceBytesPosPos) {
+  SliceOptions options{2, 4};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",
+                   this->type(), R"(["", "", "", "o", "od", "od"])", &options);
+  SliceOptions options_step{1, 5, 2};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",
+                   this->type(), R"(["", "", "o", "o", "od", "od"])", &options_step);
+  SliceOptions options_step_neg{5, 1, -2};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",
+                   this->type(), R"(["", "", "", "o", "d", "so"])", &options_step_neg);
+  options_step_neg.stop = 0;
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food","foods"])",
+                   this->type(), R"(["", "", "o", "o", "do", "so"])", &options_step_neg);
+}
+
+TYPED_TEST(TestBaseBinaryKernels, SliceBytesPosNeg) {
+  SliceOptions options{2, -1};
+  this->CheckUnary("binary_slice_bytes", R"(["", "f", "fo", "foo", "food", "foods"])",

Review Comment:
   Also, it would be nice to put some non-ASCII bytes in there as well, to check that slicing is really byte-wise.



##########
cpp/src/arrow/compute/kernels/scalar_string_ascii.cc:
##########
@@ -2409,6 +2409,172 @@ void AddAsciiStringReplaceSlice(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(func)));
 }
 
+// ----------------------------------------------------------------------
+// Slice
+
+namespace {
+struct SliceBytesTransform : StringSliceTransformBase {
+  int64_t MaxCodeunits(int64_t ninputs, int64_t input_bytes) override {
+    const SliceOptions& opt = *this->options;
+    if ((opt.start >= 0) != (opt.stop >= 0)) {
+      // If start and stop don't have the same sign, we can't guess an upper bound
+      // on the resulting slice lengths, so return a worst case estimate.
+      return input_bytes;
+    }
+    int64_t max_slice_bytes = (opt.stop - opt.start + opt.step - 1) / opt.step;
+    return std::min(input_bytes, ninputs * std::max<int64_t>(0, max_slice_bytes));
+  }
+
+  int64_t Transform(const uint8_t* input, int64_t input_string_bytes, uint8_t* output) {
+    if (options->step >= 1) {
+      return SliceForward(input, input_string_bytes, output);
+    }
+    return SliceBackward(input, input_string_bytes, output);
+  }
+
+  int64_t SliceForward(const uint8_t* input, int64_t input_string_bytes,
+                       uint8_t* output) {
+    // Slice in forward order (step > 0)
+    const SliceOptions& opt = *this->options;
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_bytes;
+    const uint8_t* begin_sliced;
+    const uint8_t* end_sliced;
+
+    if (!input_string_bytes) {
+      return 0;
+    }
+    // First, compute begin_sliced and end_sliced
+    if (opt.start >= 0) {
+      // start counting from the left
+      begin_sliced = std::min(begin + opt.start, end);
+      if (opt.stop > opt.start) {
+        // continue counting from begin_sliced
+        const int64_t length = opt.stop - opt.start;
+        end_sliced = std::min(begin_sliced + length, end);
+      } else if (opt.stop < 0) {
+        // from the end
+        end_sliced = std::max(end + opt.stop, begin_sliced);
+      } else {
+        // zero length slice
+        return 0;
+      }
+    } else {
+      // start counting from the right
+      begin_sliced = std::max(end + opt.start, begin);
+      if (opt.stop > 0) {
+        // continue counting from the left, we cannot start from begin_sliced because we
+        // don't know how many bytes are between begin and begin_sliced
+        end_sliced = std::min(begin + opt.stop, end);
+        // and therefore we also needs this
+        if (end_sliced <= begin_sliced) {
+          // zero length slice
+          return 0;
+        }
+      } else if ((opt.stop < 0) && (opt.stop > opt.start)) {
+        // stop is negative, but larger than start, so we count again from the right
+        // in some cases we can optimize this, depending on the shortest path (from end
+        // or begin_sliced), but begin_sliced and opt.start can be 'out of sync',
+        // for instance when start=-100, when the string length is only 10.
+        end_sliced = std::max(end + opt.stop, begin_sliced);
+      } else {
+        // zero length slice
+        return 0;
+      }
+    }
+
+    // Second, copy computed slice to output
+    DCHECK(begin_sliced <= end_sliced);
+    if (opt.step == 1) {
+      // fast case, where we simply can finish with a memcpy
+      std::copy(begin_sliced, end_sliced, output);
+      return end_sliced - begin_sliced;
+    }
+
+    uint8_t* dest = output;
+    const uint8_t* i = begin_sliced;
+
+    while (i < end_sliced) {
+      *dest = *i;
+      // skip step codeunits
+      i += opt.step;
+      dest++;
+    }
+    return dest - output;
+  }
+
+  int64_t SliceBackward(const uint8_t* input, int64_t input_string_bytes,
+                        uint8_t* output) {
+    // Slice in reverse order (step < 0)
+    const SliceOptions& opt = *this->options;
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_bytes;
+    const uint8_t* begin_sliced = begin;
+    const uint8_t* end_sliced = end;
+
+    if (!input_string_bytes) {
+      return 0;
+    }
+
+    if (opt.start >= 0) {
+      // +1 because begin_sliced acts as as the end of a reverse iterator
+      begin_sliced = std::min(begin + opt.start + 1, end);
+    } else {
+      // -1 because start=-1 means the last byte, which is 0 advances
+      begin_sliced = std::max(end + opt.start + 1, begin);
+    }
+    begin_sliced--;
+
+    // similar to opt.start
+    if (opt.stop >= 0) {
+      end_sliced = std::min(begin + opt.stop + 1, end);
+    } else {
+      end_sliced = std::max(end + opt.stop + 1, begin);
+    }
+    end_sliced--;
+
+    // Copy computed slice to output
+    uint8_t* dest = output;
+    const uint8_t* i = begin_sliced;
+    while (i > end_sliced) {
+      // write a single codepoint
+      *dest = *i;
+      // and skip the remainder
+      i += opt.step;
+      dest++;
+    }
+
+    return dest - output;
+  }
+};
+
+template <typename Type>
+using SliceBytes = StringTransformExec<Type, SliceBytesTransform>;
+
+}  // namespace
+
+const FunctionDoc binary_slice_bytes_doc(
+    "Slice string",
+    ("For each string in `strings`, emit the substring defined by\n"

Review Comment:
   ```suggestion
       ("For each binary string in `strings`, emit the substring defined by\n"
   ```



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

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

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