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/12 08:46:18 UTC

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

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


##########
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(),
+                                               binary_slice_bytes_doc);
+  for (const auto& ty : BaseBinaryTypes()) {

Review Comment:
   I don't think Binary Slice should support UTF-8 strings as slicing them incorrectly will return invalid UTF-8 string
   
   Eg. `\"\xc2\xa2\"` slicing this [0:1] will return ``\"\xc2\"` which is invalid UTF. 
   
   I think this should just support Binary types. Wdyt @pitrou ?
   
   Thanks!



-- 
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