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 2020/11/10 13:14:45 UTC

[GitHub] [arrow] maartenbreddels opened a new pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

maartenbreddels opened a new pull request #8621:
URL: https://github.com/apache/arrow/pull/8621


   There is one obvious loose end in this PR, which is where to generate the `std::set` based on the `TrimOptions` (now in the ctor of UTF8TrimBase). I'm not sure what the lifetime guarantees are for this object (TrimOptions), where it makes sense to initialize this set, and when an (utf8 decoding) error occurs, how/where to report this.
   
   Although this is not a costly operation, assuming people don't pass in a billion characters to trim, I do wonder what the best approach here is in general. It does not make much sense to create the `std::set` at each `Exec` call, but that is what happens now. (This also seems to happen in `TransformMatchSubstring` for creating the `prefix_table` btw.)
   
   Maybe a good place to put per-kernel pre-compute results are the `*Options` objects, but I'm not sure if that makes sense in the current architecture.
   
   Another idea is to explore alternatives to the `std::set`. It seem that (based on the TrimManyAscii benchmark), `std::unordered_set` seemed a bit slower, and simply using a linear search: `std::find(options.characters.begin(), options.characters.end(), c) != options.characters.end()` in the predicate instead of the set doesn't seem to affect performance that much.
   
   In CPython, a bloom filter is used, I could explore to see if that makes sense, but the implementation in Arrow lives under the parquet namespace.
   
   
   
   
   


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r520803595



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1252,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::set<uint32_t> codepoints;

Review comment:
       Just index the `vector<bool>` by codepoint and make `true` mean containment.




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r530997361



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1251,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> codepoints;
+
+  explicit UTF8TrimBase(TrimOptions options) : options(options) {
+    // TODO: check return / can we raise an exception here?
+    arrow::util::UTF8ForEach(options.characters, [&](uint32_t c) {
+      codepoints.resize(std::max(c + 1, static_cast<uint32_t>(codepoints.size())));
+      codepoints.at(c) = true;
+    });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);
+  }
+
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [&](uint32_t c) {
+      bool contains = codepoints[c];
+      return !contains;
+    };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+template <typename Type>
+struct UTF8Trim : UTF8TrimBase<Type, true, true, UTF8Trim<Type>> {
+  using Base = UTF8TrimBase<Type, true, true, UTF8Trim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8LTrim : UTF8TrimBase<Type, true, false, UTF8LTrim<Type>> {
+  using Base = UTF8TrimBase<Type, true, false, UTF8LTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8RTrim : UTF8TrimBase<Type, false, true, UTF8RTrim<Type>> {
+  using Base = UTF8TrimBase<Type, false, true, UTF8RTrim<Type>>;
+  using Base::Base;
+};
+
+#endif
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimWhitespaceBase : StringTransform<Type, Derived> {
+  using offset_type = typename Type::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+
+    auto predicate = [](unsigned char c) { return !IsSpaceCharacterAscii(c); };
+    const uint8_t* begin_trimmed = left ? std::find_if(begin, end, predicate) : begin;
+    if (right & (begin_trimmed < end)) {
+      std::reverse_iterator<const uint8_t*> rbegin(end);
+      std::reverse_iterator<const uint8_t*> rend(begin_trimmed);
+      end_trimmed = std::find_if(rbegin, rend, predicate).base();
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+
+template <typename Type>
+struct AsciiTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, true, AsciiTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiLTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, false, AsciiLTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiRTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, false, true, AsciiRTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> characters;
+
+  explicit AsciiTrimBase(TrimOptions options) : options(options), characters(256) {
+    std::for_each(options.characters.begin(), options.characters.end(),
+                  [&](char c) { characters[static_cast<unsigned char>(c)] = true; });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);

Review comment:
       Can `std::move(options)`

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1251,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> codepoints;
+
+  explicit UTF8TrimBase(TrimOptions options) : options(options) {
+    // TODO: check return / can we raise an exception here?
+    arrow::util::UTF8ForEach(options.characters, [&](uint32_t c) {
+      codepoints.resize(std::max(c + 1, static_cast<uint32_t>(codepoints.size())));
+      codepoints.at(c) = true;
+    });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);
+  }
+
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [&](uint32_t c) {
+      bool contains = codepoints[c];
+      return !contains;
+    };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+template <typename Type>
+struct UTF8Trim : UTF8TrimBase<Type, true, true, UTF8Trim<Type>> {
+  using Base = UTF8TrimBase<Type, true, true, UTF8Trim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8LTrim : UTF8TrimBase<Type, true, false, UTF8LTrim<Type>> {
+  using Base = UTF8TrimBase<Type, true, false, UTF8LTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8RTrim : UTF8TrimBase<Type, false, true, UTF8RTrim<Type>> {
+  using Base = UTF8TrimBase<Type, false, true, UTF8RTrim<Type>>;
+  using Base::Base;
+};
+
+#endif
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimWhitespaceBase : StringTransform<Type, Derived> {
+  using offset_type = typename Type::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+
+    auto predicate = [](unsigned char c) { return !IsSpaceCharacterAscii(c); };
+    const uint8_t* begin_trimmed = left ? std::find_if(begin, end, predicate) : begin;
+    if (right & (begin_trimmed < end)) {
+      std::reverse_iterator<const uint8_t*> rbegin(end);
+      std::reverse_iterator<const uint8_t*> rend(begin_trimmed);
+      end_trimmed = std::find_if(rbegin, rend, predicate).base();
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+
+template <typename Type>
+struct AsciiTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, true, AsciiTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiLTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, false, AsciiLTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiRTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, false, true, AsciiRTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> characters;
+
+  explicit AsciiTrimBase(TrimOptions options) : options(options), characters(256) {

Review comment:
       Can `std::move(options)`

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -428,6 +428,59 @@ TYPED_TEST(TestStringKernels, StrptimeDoesNotProvideDefaultOptions) {
   ASSERT_RAISES(Invalid, CallFunction("strptime", {input}));
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+TYPED_TEST(TestStringKernels, TrimWhitespaceUTF8) {
+  // \xe2\x80\x88 is punctuation space
+  this->CheckUnary("utf8_trim_whitespace",
+                   "[\" foo\", null, \"bar  \", \" \xe2\x80\x88 foo bar \"]",

Review comment:
       Just for the record, you can also use the "raw strings" syntax that C++ offers:
   ```c++
   R"([" foo", null, "bar  ", " \xe2\x80\x88 foo bar "])"
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1251,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> codepoints;
+
+  explicit UTF8TrimBase(TrimOptions options) : options(options) {
+    // TODO: check return / can we raise an exception here?
+    arrow::util::UTF8ForEach(options.characters, [&](uint32_t c) {
+      codepoints.resize(std::max(c + 1, static_cast<uint32_t>(codepoints.size())));
+      codepoints.at(c) = true;
+    });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);
+  }
+
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [&](uint32_t c) {
+      bool contains = codepoints[c];
+      return !contains;
+    };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+template <typename Type>
+struct UTF8Trim : UTF8TrimBase<Type, true, true, UTF8Trim<Type>> {
+  using Base = UTF8TrimBase<Type, true, true, UTF8Trim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8LTrim : UTF8TrimBase<Type, true, false, UTF8LTrim<Type>> {
+  using Base = UTF8TrimBase<Type, true, false, UTF8LTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8RTrim : UTF8TrimBase<Type, false, true, UTF8RTrim<Type>> {
+  using Base = UTF8TrimBase<Type, false, true, UTF8RTrim<Type>>;
+  using Base::Base;
+};
+
+#endif
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimWhitespaceBase : StringTransform<Type, Derived> {
+  using offset_type = typename Type::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+
+    auto predicate = [](unsigned char c) { return !IsSpaceCharacterAscii(c); };
+    const uint8_t* begin_trimmed = left ? std::find_if(begin, end, predicate) : begin;
+    if (right & (begin_trimmed < end)) {
+      std::reverse_iterator<const uint8_t*> rbegin(end);
+      std::reverse_iterator<const uint8_t*> rend(begin_trimmed);
+      end_trimmed = std::find_if(rbegin, rend, predicate).base();
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+
+template <typename Type>
+struct AsciiTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, true, AsciiTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiLTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, false, AsciiLTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiRTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, false, true, AsciiRTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> characters;

Review comment:
       Nit, but the convention is to append a trailing underscore to instance variable names (`options_`, `characters_`), so as to distinguish them from local variables.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1251,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> codepoints;
+
+  explicit UTF8TrimBase(TrimOptions options) : options(options) {
+    // TODO: check return / can we raise an exception here?

Review comment:
       You could set the status on the KernelContext in the caller.

##########
File path: cpp/src/arrow/util/utf8_util_test.cc
##########
@@ -412,5 +412,32 @@ TEST(UTF8DecodeReverse, Basics) {
   CheckInvalid("\xF0\x80\x80");
 }
 
+TEST(UTF8FindIf, Basics) {
+  auto CheckOk = [](const std::string& s, unsigned char test, int64_t offset_left,
+                    int64_t offset_right) -> void {
+    const uint8_t* begin = reinterpret_cast<const uint8_t*>(s.c_str());
+    const uint8_t* end = begin + s.length();
+    std::reverse_iterator<const uint8_t*> rbegin(end);
+    std::reverse_iterator<const uint8_t*> rend(begin);
+    const uint8_t* left;
+    const uint8_t* right;
+    auto predicate = [&](uint32_t c) { return c == test; };
+    EXPECT_TRUE(UTF8FindIf(begin, end, predicate, &left));
+    EXPECT_TRUE(UTF8FindIfReverse(begin, end, predicate, &right));
+    EXPECT_EQ(offset_left, left - begin);
+    EXPECT_EQ(offset_right, right - begin);
+    EXPECT_EQ(std::find_if(begin, end, predicate) - begin, left - begin);
+    EXPECT_EQ(std::find_if(rbegin, rend, predicate).base() - begin, right - begin);
+  };
+
+  CheckOk("aaaba", 'a', 0, 5);
+  CheckOk("aaaba", 'b', 3, 4);
+  CheckOk("aaababa", 'b', 3, 6);
+  CheckOk("aaababa", 'c', 7, 0);
+  CheckOk("a", 'a', 0, 1);
+  CheckOk("a", 'b', 1, 0);
+  CheckOk("", 'b', 0, 0);

Review comment:
       Hmm... this conveniently only checks ASCII characters. The implementation looks ok, but still.

##########
File path: cpp/src/arrow/util/utf8.h
##########
@@ -456,6 +456,67 @@ static inline bool UTF8Transform(const uint8_t* first, const uint8_t* last,
   return true;
 }
 
+template <class Predicate>
+static inline bool UTF8FindIf(const uint8_t* first, const uint8_t* last,
+                              Predicate&& predicate, const uint8_t** position) {
+  const uint8_t* i = first;
+  while (i < last) {
+    uint32_t codepoint = 0;
+    const uint8_t* current = i;
+    if (ARROW_PREDICT_FALSE(!UTF8Decode(&i, &codepoint))) {
+      return false;
+    }
+    if (predicate(codepoint)) {
+      *position = current;
+      return true;
+    }
+  }
+  *position = last;
+  return true;
+}
+
+// same semantics as std::find_if using reverse iterators when the return value
+// having the same semantics as std::reverse_iterator<..>.base()
+template <class Predicate>
+static inline bool UTF8FindIfReverse(const uint8_t* first, const uint8_t* last,
+                                     Predicate&& predicate, const uint8_t** position) {
+  const uint8_t* i = last - 1;
+  while (i >= first) {
+    uint32_t codepoint = 0;
+    const uint8_t* current = i;
+    if (ARROW_PREDICT_FALSE(!UTF8DecodeReverse(&i, &codepoint))) {
+      return false;
+    }
+    if (predicate(codepoint)) {
+      *position = current + 1;
+      return true;
+    }
+  }
+  *position = first;
+  return true;
+}
+
+template <class UnaryFunction>
+static inline bool UTF8ForEach(const uint8_t* first, const uint8_t* last,
+                               UnaryFunction&& f) {
+  const uint8_t* i = first;
+  while (i < last) {
+    uint32_t codepoint = 0;
+    if (ARROW_PREDICT_FALSE(!UTF8Decode(&i, &codepoint))) {
+      return false;
+    }
+    f(codepoint);
+  }
+  return true;
+}
+
+template <class UnaryFunction>
+static inline bool UTF8ForEach(std::string s, UnaryFunction&& f) {

Review comment:
       `const std::string& s`, since it's neither stored nor mutated?

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -428,6 +428,59 @@ TYPED_TEST(TestStringKernels, StrptimeDoesNotProvideDefaultOptions) {
   ASSERT_RAISES(Invalid, CallFunction("strptime", {input}));
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+TYPED_TEST(TestStringKernels, TrimWhitespaceUTF8) {
+  // \xe2\x80\x88 is punctuation space
+  this->CheckUnary("utf8_trim_whitespace",
+                   "[\" foo\", null, \"bar  \", \" \xe2\x80\x88 foo bar \"]",
+                   this->type(), "[\"foo\", null, \"bar\", \"foo bar\"]");
+  this->CheckUnary("utf8_rtrim_whitespace",
+                   "[\" foo\", null, \"bar  \", \" \xe2\x80\x88 foo bar \"]",
+                   this->type(), "[\" foo\", null, \"bar\", \" \xe2\x80\x88 foo bar\"]");
+  this->CheckUnary("utf8_ltrim_whitespace",
+                   "[\" foo\", null, \"bar  \", \" \xe2\x80\x88 foo bar \"]",
+                   this->type(), "[\"foo\", null, \"bar  \", \"foo bar \"]");
+}
+
+TYPED_TEST(TestStringKernels, TrimUTF8) {
+  TrimOptions options{"ȺA"};
+  this->CheckUnary("utf8_trim", "[\"ȺȺfooȺAȺ\", null, \"barȺAȺ\", \"ȺAȺfooȺAȺbarA\"]",
+                   this->type(), "[\"foo\", null, \"bar\", \"fooȺAȺbar\"]", &options);
+  this->CheckUnary("utf8_ltrim", "[\"ȺȺfooȺAȺ\", null, \"barȺAȺ\", \"ȺAȺfooȺAȺbarA\"]",
+                   this->type(), "[\"fooȺAȺ\", null, \"barȺAȺ\", \"fooȺAȺbarA\"]",
+                   &options);
+  this->CheckUnary("utf8_rtrim", "[\"ȺȺfooȺAȺ\", null, \"barȺAȺ\", \"ȺAȺfooȺAȺbarA\"]",
+                   this->type(), "[\"ȺȺfoo\", null, \"bar\", \"ȺAȺfooȺAȺbar\"]",
+                   &options);
+}
+#endif
+
+TYPED_TEST(TestStringKernels, TrimWhitespaceAscii) {
+  // \xe2\x80\x88 is punctuation space

Review comment:
       Also check regular Ascii whitespace such `\t` or `\r`?




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

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



[GitHub] [arrow] pitrou commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-724797289


   Feel free to open a JIRA about that, by the way :-)


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

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



[GitHub] [arrow] maartenbreddels commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-759276580


   ready for review @kszucs or @pitrou
   failure is fsspec (already reported on jira by joris)


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r520675172



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1252,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::set<uint32_t> codepoints;

Review comment:
       `std::set` doesn't sound like a terrific data structure for this. I would expect `std::vector<bool>` (which is [specified](https://en.cppreference.com/w/cpp/container/vector_bool) as optimizing memory footprint) to give better perf, though benchmarks are required to confirm this.




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

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



[GitHub] [arrow] maartenbreddels commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r555979073



##########
File path: cpp/src/arrow/util/utf8.h
##########
@@ -456,6 +456,67 @@ static inline bool UTF8Transform(const uint8_t* first, const uint8_t* last,
   return true;
 }
 
+template <class Predicate>
+static inline bool UTF8FindIf(const uint8_t* first, const uint8_t* last,
+                              Predicate&& predicate, const uint8_t** position) {
+  const uint8_t* i = first;
+  while (i < last) {
+    uint32_t codepoint = 0;
+    const uint8_t* current = i;
+    if (ARROW_PREDICT_FALSE(!UTF8Decode(&i, &codepoint))) {
+      return false;
+    }
+    if (predicate(codepoint)) {
+      *position = current;
+      return true;
+    }
+  }
+  *position = last;
+  return true;
+}
+
+// same semantics as std::find_if using reverse iterators when the return value
+// having the same semantics as std::reverse_iterator<..>.base()
+template <class Predicate>
+static inline bool UTF8FindIfReverse(const uint8_t* first, const uint8_t* last,
+                                     Predicate&& predicate, const uint8_t** position) {
+  const uint8_t* i = last - 1;
+  while (i >= first) {
+    uint32_t codepoint = 0;
+    const uint8_t* current = i;
+    if (ARROW_PREDICT_FALSE(!UTF8DecodeReverse(&i, &codepoint))) {
+      return false;
+    }
+    if (predicate(codepoint)) {
+      *position = current + 1;

Review comment:
       I added some more text, hope that clarifies enough. See also 
   https://stackoverflow.com/questions/14760134/why-does-removing-the-first-element-of-a-list-invalidate-rend/14760316#14760316 




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

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



[GitHub] [arrow] pitrou commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-762886294


   Ok, the PR seems ok on its own, I just added a question about some existing code.


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

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



[GitHub] [arrow] maartenbreddels commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r521422359



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1251,305 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> codepoints;
+
+  explicit UTF8TrimBase(TrimOptions options) : options(options) {
+    // TODO: check return / can we raise an exception here?
+    arrow::util::UTF8ForEach(options.characters, [&](uint32_t c) {
+      codepoints.resize(std::max(c + 1, static_cast<uint32_t>(codepoints.size())));
+      codepoints.at(c) = true;
+    });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);
+  }
+
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [&](uint32_t c) {
+      bool contains = codepoints[c];
+      return !contains;
+    };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+template <typename Type>
+struct UTF8Trim : UTF8TrimBase<Type, true, true, UTF8Trim<Type>> {
+  using Base = UTF8TrimBase<Type, true, true, UTF8Trim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8LTrim : UTF8TrimBase<Type, true, false, UTF8LTrim<Type>> {
+  using Base = UTF8TrimBase<Type, true, false, UTF8LTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8RTrim : UTF8TrimBase<Type, false, true, UTF8RTrim<Type>> {
+  using Base = UTF8TrimBase<Type, false, true, UTF8RTrim<Type>>;
+  using Base::Base;
+};
+
+#endif
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimWhitespaceBase : StringTransform<Type, Derived> {
+  using offset_type = typename Type::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+
+    auto predicate = [](unsigned char c) { return !IsSpaceCharacterAscii(c); };
+    const uint8_t* begin_trimmed = left ? std::find_if(begin, end, predicate) : begin;
+    if (right & (begin_trimmed < end)) {
+      std::reverse_iterator<const uint8_t*> rbegin(end);
+      std::reverse_iterator<const uint8_t*> rend(begin_trimmed);
+      end_trimmed = std::find_if(rbegin, rend, predicate).base();
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+
+template <typename Type>
+struct AsciiTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, true, AsciiTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiLTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, false, AsciiLTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiRTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, false, true, AsciiRTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> characters;
+
+  explicit AsciiTrimBase(TrimOptions options) : options(options), characters(256) {
+    std::for_each(options.characters.begin(), options.characters.end(),
+                  [&](char c) { characters[static_cast<unsigned char>(c)] = true; });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);
+  }
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed;
+
+    auto predicate = [&](unsigned char c) {
+      bool contains = characters[c];
+      return !contains;
+    };
+
+    begin_trimmed = left ? std::find_if(begin, end, predicate) : begin;
+    if (right & (begin_trimmed < end)) {
+      std::reverse_iterator<const uint8_t*> rbegin(end);
+      std::reverse_iterator<const uint8_t*> rend(begin_trimmed);
+      end_trimmed = std::find_if(rbegin, rend, predicate).base();
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+
+template <typename Type>
+struct AsciiTrim : AsciiTrimBase<Type, true, true, AsciiTrim<Type>> {
+  using Base = AsciiTrimBase<Type, true, true, AsciiTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct AsciiLTrim : AsciiTrimBase<Type, true, false, AsciiLTrim<Type>> {
+  using Base = AsciiTrimBase<Type, true, false, AsciiLTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct AsciiRTrim : AsciiTrimBase<Type, false, true, AsciiRTrim<Type>> {
+  using Base = AsciiTrimBase<Type, false, true, AsciiRTrim<Type>>;
+  using Base::Base;
+};
+
+const FunctionDoc utf8_trim_whitespace_doc(
+    "Trim leading and trailing whitespace characters",
+    ("For each string in `strings`, emit a string with leading and trailing whitespace "

Review comment:
       Thanks, I didn't notice, fixed.




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r551437108



##########
File path: cpp/src/arrow/util/utf8.h
##########
@@ -456,6 +456,67 @@ static inline bool UTF8Transform(const uint8_t* first, const uint8_t* last,
   return true;
 }
 
+template <class Predicate>
+static inline bool UTF8FindIf(const uint8_t* first, const uint8_t* last,
+                              Predicate&& predicate, const uint8_t** position) {
+  const uint8_t* i = first;
+  while (i < last) {
+    uint32_t codepoint = 0;
+    const uint8_t* current = i;
+    if (ARROW_PREDICT_FALSE(!UTF8Decode(&i, &codepoint))) {
+      return false;
+    }
+    if (predicate(codepoint)) {
+      *position = current;
+      return true;
+    }
+  }
+  *position = last;
+  return true;
+}
+
+// same semantics as std::find_if using reverse iterators when the return value
+// having the same semantics as std::reverse_iterator<..>.base()
+template <class Predicate>
+static inline bool UTF8FindIfReverse(const uint8_t* first, const uint8_t* last,
+                                     Predicate&& predicate, const uint8_t** position) {
+  const uint8_t* i = last - 1;
+  while (i >= first) {
+    uint32_t codepoint = 0;
+    const uint8_t* current = i;
+    if (ARROW_PREDICT_FALSE(!UTF8DecodeReverse(&i, &codepoint))) {
+      return false;
+    }
+    if (predicate(codepoint)) {
+      *position = current + 1;

Review comment:
       This is a bit weird. It returns the position to the next codepoint? The docstring should be a bit clearer about that (the current spelling is cryptic to me).

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -186,6 +172,40 @@ struct UTF8Transform {
   }
 };
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, typename Derived>
+struct UTF8Transform : StringTransform<Type, Derived> {

Review comment:
       I don't exactly understand this refactor. There's a `UTF8Transform` with a `Transform` method for utf8 kernels but no corresponding class with a `Transform` method for ascii kernels, is that right?




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r560227216



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -186,16 +172,51 @@ struct UTF8Transform {
   }
 };
 
+#ifdef ARROW_WITH_UTF8PROC
+
+// transforms per codepoint
+template <typename Type, typename Derived>
+struct StringTransformCodepoint : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    uint8_t* output_start = output;
+    if (ARROW_PREDICT_FALSE(
+            !arrow::util::UTF8Transform(input, input + input_string_ncodeunits, &output,
+                                        Derived::TransformCodepoint))) {
+      return false;
+    }
+    *output_written = static_cast<offset_type>(output - output_start);
+    return true;
+  }
+  static int64_t MaxCodepoints(offset_type input_ncodeunits) {
+    // Section 5.18 of the Unicode spec claim that the number of codepoints for case
+    // mapping can grow by a factor of 3. This means grow by a factor of 3 in bytes
+    // However, since we don't support all casings (SpecialCasing.txt) the growth
+    // is actually only at max 3/2 (as covered by the unittest).
+    // Note that rounding down the 3/2 is ok, since only codepoints encoded by
+    // two code units (even) can grow to 3 code units.
+
+    return static_cast<int64_t>(input_ncodeunits) * 3 / 2;

Review comment:
       Now that I read this again, it strikes me that this function is estimating a number of codepoints, but we're using it to allocate a number of bytes (i.e. utf-8 codeunits). Is that ok?
   
   (and/or the function naming and comment is inconsistent)




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

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



[GitHub] [arrow] maartenbreddels commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r555979359



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -186,6 +172,40 @@ struct UTF8Transform {
   }
 };
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, typename Derived>
+struct UTF8Transform : StringTransform<Type, Derived> {

Review comment:
       Yeah, that was a bad choice of name, StringTransformCodepoint is more descriptive, it's a per codepoint transformation.




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

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



[GitHub] [arrow] maartenbreddels commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-725345905


   The `std::vector<bool>` was a good idea, and indeed because of it's bit usage, the memory usage for Unicode isn't that heavy (most extreme: `0x10FFFF bits = 140kb` in case of a contiguous array implementation).
   
   Benchmarks:
   ```
   set:
   TrimManyAscii_median   28346892 ns   28345125 ns         25   558.956MB/s   35.2794M items/s
   TrimManyUtf8_median    28302644 ns   28294883 ns         25   559.949MB/s   35.3421M items/s
   
   unordered_set:
   TrimManyAscii_median   32017530 ns   32014024 ns         22   494.898MB/s   31.2363M items/s
   TrimManyUtf8_median (not run)
   
   vector<bool>
   TrimManyAscii_median   14911543 ns   14910620 ns         47   1062.58MB/s   67.0663M items/s
   TrimManyUtf8_median    16148001 ns   16146053 ns         44   981.273MB/s   61.9346M items/s
   
   bitset<256>
   TrimManyAscii_median   14304925 ns   14304010 ns         49   1107.64MB/s   69.9105M items/s
   ```
   
   
   `vector<bool>` is good enough I think, the bitset is consistently faster (5%), but I'd rather have similar code for both solutions.
   
   


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

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



[GitHub] [arrow] pitrou closed pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #8621:
URL: https://github.com/apache/arrow/pull/8621


   


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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-724695171


   https://issues.apache.org/jira/browse/ARROW-9128


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

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



[GitHub] [arrow] pitrou commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-724796243


   > Maybe a good place to put per-kernel pre-compute results are the *Options objects, but I'm not sure if that makes sense in the current architecture.
   
   I don't think the `Options` objects are the right place. Ideally the kernel state would be options-specific, otherwise we can devise a generic caching facility.


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

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



[GitHub] [arrow] maartenbreddels commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-733820473


   @pitrou this is ready for review.


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

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



[GitHub] [arrow] maartenbreddels commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r560244159



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -186,16 +172,51 @@ struct UTF8Transform {
   }
 };
 
+#ifdef ARROW_WITH_UTF8PROC
+
+// transforms per codepoint
+template <typename Type, typename Derived>
+struct StringTransformCodepoint : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    uint8_t* output_start = output;
+    if (ARROW_PREDICT_FALSE(
+            !arrow::util::UTF8Transform(input, input + input_string_ncodeunits, &output,
+                                        Derived::TransformCodepoint))) {
+      return false;
+    }
+    *output_written = static_cast<offset_type>(output - output_start);
+    return true;
+  }
+  static int64_t MaxCodepoints(offset_type input_ncodeunits) {
+    // Section 5.18 of the Unicode spec claim that the number of codepoints for case
+    // mapping can grow by a factor of 3. This means grow by a factor of 3 in bytes
+    // However, since we don't support all casings (SpecialCasing.txt) the growth
+    // is actually only at max 3/2 (as covered by the unittest).
+    // Note that rounding down the 3/2 is ok, since only codepoints encoded by
+    // two code units (even) can grow to 3 code units.
+
+    return static_cast<int64_t>(input_ncodeunits) * 3 / 2;

Review comment:
       Good catch, that was completely wrong, also slightly modified the text, this is all about units/bytes.




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

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



[GitHub] [arrow] pitrou commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-725493506


   > I guess we still need to manually add content to compute.rst?
   
   Yes, you do :-)


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

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



[GitHub] [arrow] maartenbreddels commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-753949140


   @pitrou this is ready for review


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

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



[GitHub] [arrow] maartenbreddels commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-758845894


   I agree, I'll do my best to be responsive to get this in soon!


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r521311397



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1251,305 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> codepoints;
+
+  explicit UTF8TrimBase(TrimOptions options) : options(options) {
+    // TODO: check return / can we raise an exception here?
+    arrow::util::UTF8ForEach(options.characters, [&](uint32_t c) {
+      codepoints.resize(std::max(c + 1, static_cast<uint32_t>(codepoints.size())));
+      codepoints.at(c) = true;
+    });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);
+  }
+
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [&](uint32_t c) {
+      bool contains = codepoints[c];
+      return !contains;
+    };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+template <typename Type>
+struct UTF8Trim : UTF8TrimBase<Type, true, true, UTF8Trim<Type>> {
+  using Base = UTF8TrimBase<Type, true, true, UTF8Trim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8LTrim : UTF8TrimBase<Type, true, false, UTF8LTrim<Type>> {
+  using Base = UTF8TrimBase<Type, true, false, UTF8LTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct UTF8RTrim : UTF8TrimBase<Type, false, true, UTF8RTrim<Type>> {
+  using Base = UTF8TrimBase<Type, false, true, UTF8RTrim<Type>>;
+  using Base::Base;
+};
+
+#endif
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimWhitespaceBase : StringTransform<Type, Derived> {
+  using offset_type = typename Type::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+
+    auto predicate = [](unsigned char c) { return !IsSpaceCharacterAscii(c); };
+    const uint8_t* begin_trimmed = left ? std::find_if(begin, end, predicate) : begin;
+    if (right & (begin_trimmed < end)) {
+      std::reverse_iterator<const uint8_t*> rbegin(end);
+      std::reverse_iterator<const uint8_t*> rend(begin_trimmed);
+      end_trimmed = std::find_if(rbegin, rend, predicate).base();
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+
+template <typename Type>
+struct AsciiTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, true, AsciiTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiLTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, true, false, AsciiLTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct AsciiRTrimWhitespace
+    : AsciiTrimWhitespaceBase<Type, false, true, AsciiRTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct AsciiTrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> characters;
+
+  explicit AsciiTrimBase(TrimOptions options) : options(options), characters(256) {
+    std::for_each(options.characters.begin(), options.characters.end(),
+                  [&](char c) { characters[static_cast<unsigned char>(c)] = true; });
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    TrimOptions options = State::Get(ctx);
+    Derived(options).Execute(ctx, batch, out);
+  }
+
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed;
+
+    auto predicate = [&](unsigned char c) {
+      bool contains = characters[c];
+      return !contains;
+    };
+
+    begin_trimmed = left ? std::find_if(begin, end, predicate) : begin;
+    if (right & (begin_trimmed < end)) {
+      std::reverse_iterator<const uint8_t*> rbegin(end);
+      std::reverse_iterator<const uint8_t*> rend(begin_trimmed);
+      end_trimmed = std::find_if(rbegin, rend, predicate).base();
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+};
+
+template <typename Type>
+struct AsciiTrim : AsciiTrimBase<Type, true, true, AsciiTrim<Type>> {
+  using Base = AsciiTrimBase<Type, true, true, AsciiTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct AsciiLTrim : AsciiTrimBase<Type, true, false, AsciiLTrim<Type>> {
+  using Base = AsciiTrimBase<Type, true, false, AsciiLTrim<Type>>;
+  using Base::Base;
+};
+
+template <typename Type>
+struct AsciiRTrim : AsciiTrimBase<Type, false, true, AsciiRTrim<Type>> {
+  using Base = AsciiTrimBase<Type, false, true, AsciiRTrim<Type>>;
+  using Base::Base;
+};
+
+const FunctionDoc utf8_trim_whitespace_doc(
+    "Trim leading and trailing whitespace characters",
+    ("For each string in `strings`, emit a string with leading and trailing whitespace "

Review comment:
       The other docstrings include `\n` manually at the end of each line. I suppose this is to ensure it's properly formatted in python's help (but didn't check how it looks with this PR). 
   
   (Now, I assume we could also update the python code that assembles the docstring to automatically do line wrapping with `textwrap` stdlib module, but that's probably something for another issue)




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

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



[GitHub] [arrow] kszucs commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
kszucs commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-758708026


   It'd be nice to include in the release, so ping @maartenbreddels 


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

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



[GitHub] [arrow] maartenbreddels commented on pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#issuecomment-725476725


   I've opened an issue at https://issues.apache.org/jira/browse/ARROW-10556
   
   I guess we still need to manually add content to compute.rst? 


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

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



[GitHub] [arrow] maartenbreddels commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r545867768



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1251,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::vector<bool> codepoints;
+
+  explicit UTF8TrimBase(TrimOptions options) : options(options) {
+    // TODO: check return / can we raise an exception here?

Review comment:
       I've refactored this a bit, the kernel state now include the codepoint vector, and there's where I have access to the KernelContext*.




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

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



[GitHub] [arrow] maartenbreddels commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r545752285



##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -428,6 +428,59 @@ TYPED_TEST(TestStringKernels, StrptimeDoesNotProvideDefaultOptions) {
   ASSERT_RAISES(Invalid, CallFunction("strptime", {input}));
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+TYPED_TEST(TestStringKernels, TrimWhitespaceUTF8) {
+  // \xe2\x80\x88 is punctuation space
+  this->CheckUnary("utf8_trim_whitespace",
+                   "[\" foo\", null, \"bar  \", \" \xe2\x80\x88 foo bar \"]",

Review comment:
       But that does not escape the escape chars, so you end up with:
   ```
   R"([" foo", null, "bar  ", " )"
   "\xe2\x80\x88"
   R"(foo bar "])"
   ```
   Which I also consider ugly :)




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

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



[GitHub] [arrow] maartenbreddels commented on a change in pull request #8621: ARROW-9128: [C++] Implement string space trimming kernels: trim, ltrim, and rtrim

Posted by GitBox <gi...@apache.org>.
maartenbreddels commented on a change in pull request #8621:
URL: https://github.com/apache/arrow/pull/8621#discussion_r520802117



##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -1231,6 +1252,302 @@ Result<ValueDescr> StrptimeResolve(KernelContext* ctx, const std::vector<ValueDe
   return Status::Invalid("strptime does not provide default StrptimeOptions");
 }
 
+#ifdef ARROW_WITH_UTF8PROC
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimWhitespaceBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  bool Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                 uint8_t* output, offset_type* output_written) {
+    const uint8_t* begin = input;
+    const uint8_t* end = input + input_string_ncodeunits;
+    const uint8_t* end_trimmed = end;
+    const uint8_t* begin_trimmed = begin;
+
+    auto predicate = [](uint32_t c) { return !IsSpaceCharacterUnicode(c); };
+    if (left && !ARROW_PREDICT_TRUE(
+                    arrow::util::UTF8FindIf(begin, end, predicate, &begin_trimmed))) {
+      return false;
+    }
+    if (right && (begin_trimmed < end)) {
+      if (!ARROW_PREDICT_TRUE(arrow::util::UTF8FindIfReverse(begin_trimmed, end,
+                                                             predicate, &end_trimmed))) {
+        return false;
+      }
+    }
+    std::copy(begin_trimmed, end_trimmed, output);
+    *output_written = static_cast<offset_type>(end_trimmed - begin_trimmed);
+    return true;
+  }
+  void Execute(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    EnsureLookupTablesFilled();
+    Base::Execute(ctx, batch, out);
+  }
+};
+
+template <typename Type>
+struct UTF8TrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, true, UTF8TrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8LTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, true, false, UTF8LTrimWhitespace<Type>> {};
+
+template <typename Type>
+struct UTF8RTrimWhitespace
+    : UTF8TrimWhitespaceBase<Type, false, true, UTF8RTrimWhitespace<Type>> {};
+
+template <typename Type, bool left, bool right, typename Derived>
+struct UTF8TrimBase : StringTransform<Type, Derived> {
+  using Base = StringTransform<Type, Derived>;
+  using offset_type = typename Base::offset_type;
+  using State = OptionsWrapper<TrimOptions>;
+  TrimOptions options;
+  std::set<uint32_t> codepoints;

Review comment:
       We need to test if a codepoint is present in a 'set' of codepoints (specified by options.characters), I don't see how an `std::vector<bool>` is useful here, maybe I'm not seeing it.




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

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