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/06/25 09:33:30 UTC

[GitHub] [arrow] pitrou commented on a change in pull request #7449: ARROW-9133: [C++] Add utf8_upper and utf8_lower

pitrou commented on a change in pull request #7449:
URL: https://github.com/apache/arrow/pull/7449#discussion_r445417901



##########
File path: cpp/src/arrow/compute/kernels/scalar_string_benchmark.cc
##########
@@ -52,8 +55,18 @@ static void AsciiUpper(benchmark::State& state) {
   UnaryStringBenchmark(state, "ascii_upper");
 }
 
+static void Utf8Upper(benchmark::State& state) {
+  UnaryStringBenchmark(state, "utf8_upper", true);

Review comment:
       Note that `RandomArrayGenerator::String` generates an array of pure Ascii characters between `A` and `z`. 
   Perhaps we need two sets of benchmarks:
   * one with pure Ascii values
   * one with partly non-Ascii values

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -30,6 +31,124 @@ namespace internal {
 
 namespace {
 
+// lookup tables
+std::vector<uint32_t> lut_upper_codepoint;
+std::vector<uint32_t> lut_lower_codepoint;
+std::once_flag flag_case_luts;
+
+constexpr uint32_t REPLACEMENT_CHAR =
+    '?';  // the proper replacement char would be the 0xFFFD codepoint, but that can
+          // increase string length by a factor of 3
+constexpr int MAX_CODEPOINT_LUT = 0xffff;  // up to this codepoint is in a lookup table
+
+static inline void utf8_encode(uint8_t*& str, uint32_t codepoint) {
+  if (codepoint < 0x80) {
+    *str++ = codepoint;
+  } else if (codepoint < 0x800) {
+    *str++ = 0xC0 + (codepoint >> 6);
+    *str++ = 0x80 + (codepoint & 0x3F);
+  } else if (codepoint < 0x10000) {
+    *str++ = 0xE0 + (codepoint >> 12);
+    *str++ = 0x80 + ((codepoint >> 6) & 0x3F);
+    *str++ = 0x80 + (codepoint & 0x3F);
+  } else if (codepoint < 0x200000) {
+    *str++ = 0xF0 + (codepoint >> 18);
+    *str++ = 0x80 + ((codepoint >> 12) & 0x3F);
+    *str++ = 0x80 + ((codepoint >> 6) & 0x3F);
+    *str++ = 0x80 + (codepoint & 0x3F);
+  } else {
+    *str++ = codepoint;
+  }
+}
+
+static inline bool utf8_is_continuation(const uint8_t codeunit) {
+  return (codeunit & 0xC0) == 0x80;  // upper two bits should be 10
+}
+
+static inline uint32_t utf8_decode(const uint8_t*& str, int64_t& length) {
+  if (*str < 0x80) {  //
+    length -= 1;
+    return *str++;
+  } else if (*str < 0xC0) {  // invalid non-ascii char
+    length -= 1;
+    str++;
+    return REPLACEMENT_CHAR;

Review comment:
       Hmm, I don't really agree with this... If there's some invalid input, we should bail out with `Status::Invalid`, IMO.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_benchmark.cc
##########
@@ -41,7 +42,9 @@ static void UnaryStringBenchmark(benchmark::State& state, const std::string& fun
     ABORT_NOT_OK(CallFunction(func_name, {values}));
   }
   state.SetItemsProcessed(state.iterations() * array_length);
-  state.SetBytesProcessed(state.iterations() * values->data()->buffers[2]->size());
+  state.SetBytesProcessed(state.iterations() *
+                          ((touches_offsets ? values->data()->buffers[1]->size() : 0) +

Review comment:
       Hmm, this looks a bit pedantic and counter-productive to me. We want Ascii and UTF8 numbers to be directly comparable, so let's keep the original calculation.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -30,6 +31,124 @@ namespace internal {
 
 namespace {
 
+// lookup tables
+std::vector<uint32_t> lut_upper_codepoint;
+std::vector<uint32_t> lut_lower_codepoint;
+std::once_flag flag_case_luts;
+
+constexpr uint32_t REPLACEMENT_CHAR =
+    '?';  // the proper replacement char would be the 0xFFFD codepoint, but that can
+          // increase string length by a factor of 3
+constexpr int MAX_CODEPOINT_LUT = 0xffff;  // up to this codepoint is in a lookup table
+
+static inline void utf8_encode(uint8_t*& str, uint32_t codepoint) {

Review comment:
       By the way, all those helper functions (encoding/decoding) may also go into `arrow/util/utf8.h`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -68,6 +76,64 @@ TYPED_TEST(TestStringKernels, AsciiLower) {
                    this->string_type(), "[\"aaazzæÆ&\", null, \"\", \"bbb\"]");
 }
 
+TEST(TestStringKernels, Utf8Upper32bitGrowth) {

Review comment:
       I'm not convinced it's a good idea to consume more than 2GB RAM in a test. We have something called `LARGE_MEMORY_TEST` for such tests (you can grep for it), though I'm not sure they get exercised on a regular basis.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -81,5 +147,40 @@ TYPED_TEST(TestStringKernels, StrptimeDoesNotProvideDefaultOptions) {
   ASSERT_RAISES(Invalid, CallFunction("strptime", {input}));
 }
 
+TEST(TestStringKernels, UnicodeLibraryAssumptions) {
+  uint8_t output[4];
+  for (utf8proc_int32_t codepoint = 0x100; codepoint < 0x110000; codepoint++) {
+    utf8proc_ssize_t encoded_nbytes = utf8proc_encode_char(codepoint, output);
+    utf8proc_int32_t codepoint_upper = utf8proc_toupper(codepoint);
+    utf8proc_ssize_t encoded_nbytes_upper = utf8proc_encode_char(codepoint_upper, output);
+    if (encoded_nbytes == 2) {
+      EXPECT_LE(encoded_nbytes_upper, 3)
+          << "Expected the upper case codepoint for a 2 byte encoded codepoint to be "
+             "encoded in maximum 3 bytes, not "
+          << encoded_nbytes_upper;
+    }
+    if (encoded_nbytes == 3) {

Review comment:
       And what about `encoded_nbytes == 4`?

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {

Review comment:
       Could make this a separate helper function, for clarity.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =
+          input.buffers[2]->data() + input_boxed.value_offset(0);
+      offset_type input_ncodeunits = input_boxed.total_values_length();
+      offset_type input_nstrings = (offset_type)input.length;
+
+      // 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.
+
+      int64_t output_ncodeunits_max = ((int64_t)input_ncodeunits) * 3 / 2;
+      if (output_ncodeunits_max > std::numeric_limits<offset_type>::max()) {
+        ctx->SetStatus(Status::CapacityError(
+            "Result might not fit in a 32bit utf8 array, convert to large_utf8"));
+        return;
+      }
+
+      KERNEL_RETURN_IF_ERROR(
+          ctx, ctx->Allocate(output_ncodeunits_max).Value(&output->buffers[2]));
+      // We could reuse the buffer if it is all ascii, benchmarking showed this not to
+      // matter
+      // output->buffers[1] = input.buffers[1];
+      KERNEL_RETURN_IF_ERROR(ctx,
+                             ctx->Allocate((input_nstrings + 1) * sizeof(offset_type))
+                                 .Value(&output->buffers[1]));
+      utf8proc_uint8_t* output_str = output->buffers[2]->mutable_data();
+      offset_type* output_string_offsets = output->GetMutableValues<offset_type>(1);
+      offset_type output_ncodeunits = 0;
+
+      offset_type output_string_offset = 0;
+      *output_string_offsets = output_string_offset;
+      offset_type input_string_first_offset = input_string_offsets[0];
+      for (int64_t i = 0; i < input_nstrings; i++) {
+        offset_type input_string_offset =
+            input_string_offsets[i] - input_string_first_offset;
+        offset_type input_string_end =
+            input_string_offsets[i + 1] - input_string_first_offset;
+        offset_type input_string_ncodeunits = input_string_end - input_string_offset;
+        offset_type encoded_nbytes = DerivedClass::Transform(
+            input_str + input_string_offset, input_string_ncodeunits,
+            output_str + output_ncodeunits);
+        output_ncodeunits += encoded_nbytes;
+        output_string_offsets[i + 1] = output_ncodeunits;
+      }
+
+      // trim the codepoint buffer, since we allocated too much
+      KERNEL_RETURN_IF_ERROR(
+          ctx,
+          output->buffers[2]->CopySlice(0, output_ncodeunits).Value(&output->buffers[2]));

Review comment:
       Hmm, we should be able to resize the buffer instead... except that `KernelContext::Allocate` doesn't return a `ResizableBuffer`. @wesm Do you think we can change that?

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =
+          input.buffers[2]->data() + input_boxed.value_offset(0);
+      offset_type input_ncodeunits = input_boxed.total_values_length();
+      offset_type input_nstrings = (offset_type)input.length;
+
+      // 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.
+
+      int64_t output_ncodeunits_max = ((int64_t)input_ncodeunits) * 3 / 2;
+      if (output_ncodeunits_max > std::numeric_limits<offset_type>::max()) {
+        ctx->SetStatus(Status::CapacityError(
+            "Result might not fit in a 32bit utf8 array, convert to large_utf8"));
+        return;
+      }
+
+      KERNEL_RETURN_IF_ERROR(
+          ctx, ctx->Allocate(output_ncodeunits_max).Value(&output->buffers[2]));
+      // We could reuse the buffer if it is all ascii, benchmarking showed this not to
+      // matter
+      // output->buffers[1] = input.buffers[1];
+      KERNEL_RETURN_IF_ERROR(ctx,
+                             ctx->Allocate((input_nstrings + 1) * sizeof(offset_type))
+                                 .Value(&output->buffers[1]));
+      utf8proc_uint8_t* output_str = output->buffers[2]->mutable_data();
+      offset_type* output_string_offsets = output->GetMutableValues<offset_type>(1);
+      offset_type output_ncodeunits = 0;
+
+      offset_type output_string_offset = 0;
+      *output_string_offsets = output_string_offset;
+      offset_type input_string_first_offset = input_string_offsets[0];
+      for (int64_t i = 0; i < input_nstrings; i++) {
+        offset_type input_string_offset =
+            input_string_offsets[i] - input_string_first_offset;
+        offset_type input_string_end =
+            input_string_offsets[i + 1] - input_string_first_offset;
+        offset_type input_string_ncodeunits = input_string_end - input_string_offset;
+        offset_type encoded_nbytes = DerivedClass::Transform(
+            input_str + input_string_offset, input_string_ncodeunits,
+            output_str + output_ncodeunits);
+        output_ncodeunits += encoded_nbytes;
+        output_string_offsets[i + 1] = output_ncodeunits;
+      }
+
+      // trim the codepoint buffer, since we allocated too much
+      KERNEL_RETURN_IF_ERROR(
+          ctx,
+          output->buffers[2]->CopySlice(0, output_ncodeunits).Value(&output->buffers[2]));
+    } else {
+      const auto& input = checked_cast<const BaseBinaryScalar&>(*batch[0].scalar());
+      auto result = checked_pointer_cast<BaseBinaryScalar>(MakeNullScalar(out->type()));
+      if (input.is_valid) {
+        result->is_valid = true;
+        offset_type data_nbytes = (offset_type)input.value->size();
+        // See note above in the Array version explaining the 3 / 2
+        KERNEL_RETURN_IF_ERROR(ctx,
+                               ctx->Allocate(data_nbytes * 3 / 2).Value(&result->value));
+        offset_type encoded_nbytes = DerivedClass::Transform(
+            input.value->data(), data_nbytes, result->value->mutable_data());
+        KERNEL_RETURN_IF_ERROR(
+            ctx, result->value->CopySlice(0, encoded_nbytes).Value(&result->value));
+      }
+      out->value = result;
+    }
+  }
+};
+
+template <typename Type>
+struct Utf8Upper : Utf8Transform<Type, Utf8Upper> {
+  inline static uint32_t TransformCodepoint(char32_t codepoint) {

Review comment:
       Why `char32_t`? We're using `uint32_t` elsewhere.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -30,6 +31,124 @@ namespace internal {
 
 namespace {
 
+// lookup tables
+std::vector<uint32_t> lut_upper_codepoint;
+std::vector<uint32_t> lut_lower_codepoint;
+std::once_flag flag_case_luts;
+
+constexpr uint32_t REPLACEMENT_CHAR =
+    '?';  // the proper replacement char would be the 0xFFFD codepoint, but that can
+          // increase string length by a factor of 3
+constexpr int MAX_CODEPOINT_LUT = 0xffff;  // up to this codepoint is in a lookup table
+
+static inline void utf8_encode(uint8_t*& str, uint32_t codepoint) {
+  if (codepoint < 0x80) {
+    *str++ = codepoint;
+  } else if (codepoint < 0x800) {
+    *str++ = 0xC0 + (codepoint >> 6);
+    *str++ = 0x80 + (codepoint & 0x3F);
+  } else if (codepoint < 0x10000) {
+    *str++ = 0xE0 + (codepoint >> 12);
+    *str++ = 0x80 + ((codepoint >> 6) & 0x3F);
+    *str++ = 0x80 + (codepoint & 0x3F);
+  } else if (codepoint < 0x200000) {
+    *str++ = 0xF0 + (codepoint >> 18);
+    *str++ = 0x80 + ((codepoint >> 12) & 0x3F);
+    *str++ = 0x80 + ((codepoint >> 6) & 0x3F);
+    *str++ = 0x80 + (codepoint & 0x3F);
+  } else {
+    *str++ = codepoint;

Review comment:
       Invalid characters should not be emitted silently.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -30,6 +31,124 @@ namespace internal {
 
 namespace {
 
+// lookup tables
+std::vector<uint32_t> lut_upper_codepoint;
+std::vector<uint32_t> lut_lower_codepoint;
+std::once_flag flag_case_luts;
+
+constexpr uint32_t REPLACEMENT_CHAR =
+    '?';  // the proper replacement char would be the 0xFFFD codepoint, but that can
+          // increase string length by a factor of 3
+constexpr int MAX_CODEPOINT_LUT = 0xffff;  // up to this codepoint is in a lookup table

Review comment:
       For constants, the recommended spelling would be e.g. `constexpr uint32_t kMaxCodepointLookup`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =
+          input.buffers[2]->data() + input_boxed.value_offset(0);
+      offset_type input_ncodeunits = input_boxed.total_values_length();
+      offset_type input_nstrings = (offset_type)input.length;
+
+      // 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.
+
+      int64_t output_ncodeunits_max = ((int64_t)input_ncodeunits) * 3 / 2;
+      if (output_ncodeunits_max > std::numeric_limits<offset_type>::max()) {
+        ctx->SetStatus(Status::CapacityError(
+            "Result might not fit in a 32bit utf8 array, convert to large_utf8"));
+        return;
+      }
+
+      KERNEL_RETURN_IF_ERROR(
+          ctx, ctx->Allocate(output_ncodeunits_max).Value(&output->buffers[2]));
+      // We could reuse the buffer if it is all ascii, benchmarking showed this not to
+      // matter
+      // output->buffers[1] = input.buffers[1];
+      KERNEL_RETURN_IF_ERROR(ctx,
+                             ctx->Allocate((input_nstrings + 1) * sizeof(offset_type))
+                                 .Value(&output->buffers[1]));
+      utf8proc_uint8_t* output_str = output->buffers[2]->mutable_data();
+      offset_type* output_string_offsets = output->GetMutableValues<offset_type>(1);
+      offset_type output_ncodeunits = 0;
+
+      offset_type output_string_offset = 0;
+      *output_string_offsets = output_string_offset;
+      offset_type input_string_first_offset = input_string_offsets[0];
+      for (int64_t i = 0; i < input_nstrings; i++) {
+        offset_type input_string_offset =
+            input_string_offsets[i] - input_string_first_offset;
+        offset_type input_string_end =
+            input_string_offsets[i + 1] - input_string_first_offset;
+        offset_type input_string_ncodeunits = input_string_end - input_string_offset;
+        offset_type encoded_nbytes = DerivedClass::Transform(
+            input_str + input_string_offset, input_string_ncodeunits,
+            output_str + output_ncodeunits);
+        output_ncodeunits += encoded_nbytes;
+        output_string_offsets[i + 1] = output_ncodeunits;
+      }
+
+      // trim the codepoint buffer, since we allocated too much
+      KERNEL_RETURN_IF_ERROR(
+          ctx,
+          output->buffers[2]->CopySlice(0, output_ncodeunits).Value(&output->buffers[2]));
+    } else {
+      const auto& input = checked_cast<const BaseBinaryScalar&>(*batch[0].scalar());
+      auto result = checked_pointer_cast<BaseBinaryScalar>(MakeNullScalar(out->type()));
+      if (input.is_valid) {
+        result->is_valid = true;
+        offset_type data_nbytes = (offset_type)input.value->size();
+        // See note above in the Array version explaining the 3 / 2
+        KERNEL_RETURN_IF_ERROR(ctx,
+                               ctx->Allocate(data_nbytes * 3 / 2).Value(&result->value));
+        offset_type encoded_nbytes = DerivedClass::Transform(
+            input.value->data(), data_nbytes, result->value->mutable_data());
+        KERNEL_RETURN_IF_ERROR(
+            ctx, result->value->CopySlice(0, encoded_nbytes).Value(&result->value));
+      }
+      out->value = result;
+    }
+  }
+};
+
+template <typename Type>
+struct Utf8Upper : Utf8Transform<Type, Utf8Upper> {

Review comment:
       Why is this templated on Type?

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -68,6 +76,64 @@ TYPED_TEST(TestStringKernels, AsciiLower) {
                    this->string_type(), "[\"aaazzæÆ&\", null, \"\", \"bbb\"]");
 }
 
+TEST(TestStringKernels, Utf8Upper32bitGrowth) {
+  std::string str(0xffff, 'a');
+  arrow::StringBuilder builder;
+  // 0x7fff * 0xffff is the max a 32 bit string array can hold
+  // since the utf8_upper kernel can grow it by 3/2, the max we should accept is is
+  // 0x7fff * 0xffff * 2/3 = 0x5555 * 0xffff, so this should give us a CapacityError
+  for (int64_t i = 0; i < 0x5556; i++) {
+    ASSERT_OK(builder.Append(str));
+  }
+  std::shared_ptr<arrow::Array> array;
+  arrow::Status st = builder.Finish(&array);
+  const FunctionOptions* options = nullptr;
+  EXPECT_RAISES_WITH_MESSAGE_THAT(CapacityError,
+                                  testing::HasSubstr("Result might not fit"),
+                                  CallFunction("utf8_upper", {array}, options));
+}
+
+TYPED_TEST(TestStringKernels, Utf8Upper) {
+  this->CheckUnary("utf8_upper", "[\"aAazZæÆ&\", null, \"\", \"b\"]", this->string_type(),
+                   "[\"AAAZZÆÆ&\", null, \"\", \"B\"]");
+
+  // test varying encoding lenghts and thus changing indices/offsets
+  this->CheckUnary("utf8_upper", "[\"ɑɽⱤoW\", null, \"ıI\", \"b\"]", this->string_type(),
+                   "[\"ⱭⱤⱤOW\", null, \"II\", \"B\"]");
+
+  // ῦ to Υ͂ not supported
+  // this->CheckUnary("utf8_upper", "[\"ῦɐɜʞȿ\"]", this->string_type(),
+  // "[\"Υ͂ⱯꞫꞰⱾ\"]");
+
+  // test maximum buffer growth
+  this->CheckUnary("utf8_upper", "[\"ɑɑɑɑ\"]", this->string_type(), "[\"ⱭⱭⱭⱭ\"]");
+
+  // Test replacing invalid data by ? (ὖ == \xe1\xbd\x96)

Review comment:
       Why do you mention `ὖ` in this comment?

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -30,6 +31,124 @@ namespace internal {
 
 namespace {
 
+// lookup tables
+std::vector<uint32_t> lut_upper_codepoint;
+std::vector<uint32_t> lut_lower_codepoint;
+std::once_flag flag_case_luts;
+
+constexpr uint32_t REPLACEMENT_CHAR =
+    '?';  // the proper replacement char would be the 0xFFFD codepoint, but that can
+          // increase string length by a factor of 3
+constexpr int MAX_CODEPOINT_LUT = 0xffff;  // up to this codepoint is in a lookup table
+
+static inline void utf8_encode(uint8_t*& str, uint32_t codepoint) {

Review comment:
       Also, it disallows non-const references. Instead, you may take a pointer argument.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -30,6 +31,124 @@ namespace internal {
 
 namespace {
 
+// lookup tables
+std::vector<uint32_t> lut_upper_codepoint;
+std::vector<uint32_t> lut_lower_codepoint;
+std::once_flag flag_case_luts;
+
+constexpr uint32_t REPLACEMENT_CHAR =
+    '?';  // the proper replacement char would be the 0xFFFD codepoint, but that can
+          // increase string length by a factor of 3
+constexpr int MAX_CODEPOINT_LUT = 0xffff;  // up to this codepoint is in a lookup table
+
+static inline void utf8_encode(uint8_t*& str, uint32_t codepoint) {

Review comment:
       Our coding style mandates CamelCase except for trivial accessors.

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -68,6 +76,64 @@ TYPED_TEST(TestStringKernels, AsciiLower) {
                    this->string_type(), "[\"aaazzæÆ&\", null, \"\", \"bbb\"]");
 }
 
+TEST(TestStringKernels, Utf8Upper32bitGrowth) {
+  std::string str(0xffff, 'a');
+  arrow::StringBuilder builder;
+  // 0x7fff * 0xffff is the max a 32 bit string array can hold
+  // since the utf8_upper kernel can grow it by 3/2, the max we should accept is is
+  // 0x7fff * 0xffff * 2/3 = 0x5555 * 0xffff, so this should give us a CapacityError
+  for (int64_t i = 0; i < 0x5556; i++) {
+    ASSERT_OK(builder.Append(str));
+  }
+  std::shared_ptr<arrow::Array> array;
+  arrow::Status st = builder.Finish(&array);
+  const FunctionOptions* options = nullptr;
+  EXPECT_RAISES_WITH_MESSAGE_THAT(CapacityError,
+                                  testing::HasSubstr("Result might not fit"),
+                                  CallFunction("utf8_upper", {array}, options));
+}
+
+TYPED_TEST(TestStringKernels, Utf8Upper) {
+  this->CheckUnary("utf8_upper", "[\"aAazZæÆ&\", null, \"\", \"b\"]", this->string_type(),
+                   "[\"AAAZZÆÆ&\", null, \"\", \"B\"]");
+
+  // test varying encoding lenghts and thus changing indices/offsets
+  this->CheckUnary("utf8_upper", "[\"ɑɽⱤoW\", null, \"ıI\", \"b\"]", this->string_type(),
+                   "[\"ⱭⱤⱤOW\", null, \"II\", \"B\"]");
+
+  // ῦ to Υ͂ not supported
+  // this->CheckUnary("utf8_upper", "[\"ῦɐɜʞȿ\"]", this->string_type(),
+  // "[\"Υ͂ⱯꞫꞰⱾ\"]");
+
+  // test maximum buffer growth
+  this->CheckUnary("utf8_upper", "[\"ɑɑɑɑ\"]", this->string_type(), "[\"ⱭⱭⱭⱭ\"]");
+
+  // Test replacing invalid data by ? (ὖ == \xe1\xbd\x96)
+  this->CheckUnary("utf8_upper", "[\"ɑa\xFFɑ\", \"ɽ\xe1\xbdɽaa\"]", this->string_type(),
+                   "[\"ⱭA?Ɑ\", \"Ɽ?ⱤAA\"]");
+}
+
+TYPED_TEST(TestStringKernels, Utf8Lower) {
+  this->CheckUnary("utf8_lower", "[\"aAazZæÆ&\", null, \"\", \"b\"]", this->string_type(),
+                   "[\"aaazzææ&\", null, \"\", \"b\"]");
+
+  // test varying encoding lenghts and thus changing indices/offsets
+  this->CheckUnary("utf8_lower", "[\"ⱭɽⱤoW\", null, \"ıI\", \"B\"]", this->string_type(),
+                   "[\"ɑɽɽow\", null, \"ıi\", \"b\"]");
+
+  // ῦ to Υ͂ is not supported, but in principle the reverse is, but it would need
+  // normalization
+  // this->CheckUnary("utf8_lower", "[\"Υ͂ⱯꞫꞰⱾ\"]", this->string_type(),
+  // "[\"ῦɐɜʞȿ\"]");
+
+  // test maximum buffer growth
+  this->CheckUnary("utf8_lower", "[\"ȺȺȺȺ\"]", this->string_type(), "[\"ⱥⱥⱥⱥ\"]");
+
+  // Test replacing invalid data by ? (ὖ == \xe1\xbd\x96)

Review comment:
       Why do you mention `ὖ` in this comment?

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =
+          input.buffers[2]->data() + input_boxed.value_offset(0);
+      offset_type input_ncodeunits = input_boxed.total_values_length();
+      offset_type input_nstrings = (offset_type)input.length;

Review comment:
       `static_cast<offset_type>(input.length)`

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =
+          input.buffers[2]->data() + input_boxed.value_offset(0);
+      offset_type input_ncodeunits = input_boxed.total_values_length();
+      offset_type input_nstrings = (offset_type)input.length;
+
+      // 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.
+
+      int64_t output_ncodeunits_max = ((int64_t)input_ncodeunits) * 3 / 2;
+      if (output_ncodeunits_max > std::numeric_limits<offset_type>::max()) {
+        ctx->SetStatus(Status::CapacityError(
+            "Result might not fit in a 32bit utf8 array, convert to large_utf8"));
+        return;
+      }
+
+      KERNEL_RETURN_IF_ERROR(
+          ctx, ctx->Allocate(output_ncodeunits_max).Value(&output->buffers[2]));
+      // We could reuse the buffer if it is all ascii, benchmarking showed this not to
+      // matter
+      // output->buffers[1] = input.buffers[1];
+      KERNEL_RETURN_IF_ERROR(ctx,
+                             ctx->Allocate((input_nstrings + 1) * sizeof(offset_type))
+                                 .Value(&output->buffers[1]));
+      utf8proc_uint8_t* output_str = output->buffers[2]->mutable_data();
+      offset_type* output_string_offsets = output->GetMutableValues<offset_type>(1);
+      offset_type output_ncodeunits = 0;
+
+      offset_type output_string_offset = 0;
+      *output_string_offsets = output_string_offset;
+      offset_type input_string_first_offset = input_string_offsets[0];
+      for (int64_t i = 0; i < input_nstrings; i++) {
+        offset_type input_string_offset =
+            input_string_offsets[i] - input_string_first_offset;
+        offset_type input_string_end =
+            input_string_offsets[i + 1] - input_string_first_offset;
+        offset_type input_string_ncodeunits = input_string_end - input_string_offset;
+        offset_type encoded_nbytes = DerivedClass::Transform(
+            input_str + input_string_offset, input_string_ncodeunits,
+            output_str + output_ncodeunits);
+        output_ncodeunits += encoded_nbytes;
+        output_string_offsets[i + 1] = output_ncodeunits;
+      }
+
+      // trim the codepoint buffer, since we allocated too much
+      KERNEL_RETURN_IF_ERROR(
+          ctx,
+          output->buffers[2]->CopySlice(0, output_ncodeunits).Value(&output->buffers[2]));
+    } else {
+      const auto& input = checked_cast<const BaseBinaryScalar&>(*batch[0].scalar());
+      auto result = checked_pointer_cast<BaseBinaryScalar>(MakeNullScalar(out->type()));
+      if (input.is_valid) {
+        result->is_valid = true;
+        offset_type data_nbytes = (offset_type)input.value->size();

Review comment:
       `static_cast<offset_type>(...)`

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =
+          input.buffers[2]->data() + input_boxed.value_offset(0);
+      offset_type input_ncodeunits = input_boxed.total_values_length();
+      offset_type input_nstrings = (offset_type)input.length;
+
+      // 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.
+
+      int64_t output_ncodeunits_max = ((int64_t)input_ncodeunits) * 3 / 2;
+      if (output_ncodeunits_max > std::numeric_limits<offset_type>::max()) {
+        ctx->SetStatus(Status::CapacityError(
+            "Result might not fit in a 32bit utf8 array, convert to large_utf8"));
+        return;
+      }
+
+      KERNEL_RETURN_IF_ERROR(
+          ctx, ctx->Allocate(output_ncodeunits_max).Value(&output->buffers[2]));
+      // We could reuse the buffer if it is all ascii, benchmarking showed this not to
+      // matter
+      // output->buffers[1] = input.buffers[1];
+      KERNEL_RETURN_IF_ERROR(ctx,
+                             ctx->Allocate((input_nstrings + 1) * sizeof(offset_type))
+                                 .Value(&output->buffers[1]));
+      utf8proc_uint8_t* output_str = output->buffers[2]->mutable_data();
+      offset_type* output_string_offsets = output->GetMutableValues<offset_type>(1);
+      offset_type output_ncodeunits = 0;
+
+      offset_type output_string_offset = 0;
+      *output_string_offsets = output_string_offset;
+      offset_type input_string_first_offset = input_string_offsets[0];
+      for (int64_t i = 0; i < input_nstrings; i++) {
+        offset_type input_string_offset =
+            input_string_offsets[i] - input_string_first_offset;
+        offset_type input_string_end =
+            input_string_offsets[i + 1] - input_string_first_offset;
+        offset_type input_string_ncodeunits = input_string_end - input_string_offset;

Review comment:
       Why not simply call `GetView` instead?
   ```c++
   const auto input = boxed_input.GetView(i);
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =

Review comment:
       Just use `uint8_t`.
   Also, you should be able to write:
   ```c++
   const uint8_t* input_str = input_boxed->raw_value_offsets();
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_string_test.cc
##########
@@ -81,5 +147,40 @@ TYPED_TEST(TestStringKernels, StrptimeDoesNotProvideDefaultOptions) {
   ASSERT_RAISES(Invalid, CallFunction("strptime", {input}));
 }
 
+TEST(TestStringKernels, UnicodeLibraryAssumptions) {
+  uint8_t output[4];
+  for (utf8proc_int32_t codepoint = 0x100; codepoint < 0x110000; codepoint++) {
+    utf8proc_ssize_t encoded_nbytes = utf8proc_encode_char(codepoint, output);
+    utf8proc_int32_t codepoint_upper = utf8proc_toupper(codepoint);
+    utf8proc_ssize_t encoded_nbytes_upper = utf8proc_encode_char(codepoint_upper, output);
+    if (encoded_nbytes == 2) {
+      EXPECT_LE(encoded_nbytes_upper, 3)
+          << "Expected the upper case codepoint for a 2 byte encoded codepoint to be "
+             "encoded in maximum 3 bytes, not "
+          << encoded_nbytes_upper;
+    }
+    if (encoded_nbytes == 3) {

Review comment:
       (also, what's the point of two different conditionals if you're doing the same thing inside?)

##########
File path: cpp/src/arrow/compute/kernels/scalar_string.cc
##########
@@ -39,6 +158,121 @@ struct AsciiLength {
   }
 };
 
+template <typename Type, template <typename> class Derived>
+struct Utf8Transform {
+  using offset_type = typename Type::offset_type;
+  using DerivedClass = Derived<Type>;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+
+  static offset_type Transform(const uint8_t* input, offset_type input_string_ncodeunits,
+                               uint8_t* output) {
+    uint8_t* dest = output;
+    utf8_transform(input, input + input_string_ncodeunits, dest,
+                   DerivedClass::TransformCodepoint);
+    return (offset_type)(dest - output);
+  }
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      std::call_once(flag_case_luts, []() {
+        lut_upper_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        lut_lower_codepoint.reserve(MAX_CODEPOINT_LUT + 1);
+        for (int i = 0; i <= MAX_CODEPOINT_LUT; i++) {
+          lut_upper_codepoint.push_back(utf8proc_toupper(i));
+          lut_lower_codepoint.push_back(utf8proc_tolower(i));
+        }
+      });
+      const ArrayData& input = *batch[0].array();
+      ArrayType input_boxed(batch[0].array());
+      ArrayData* output = out->mutable_array();
+
+      offset_type const* input_string_offsets = input.GetValues<offset_type>(1);
+      utf8proc_uint8_t const* input_str =
+          input.buffers[2]->data() + input_boxed.value_offset(0);
+      offset_type input_ncodeunits = input_boxed.total_values_length();
+      offset_type input_nstrings = (offset_type)input.length;
+
+      // 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.
+
+      int64_t output_ncodeunits_max = ((int64_t)input_ncodeunits) * 3 / 2;
+      if (output_ncodeunits_max > std::numeric_limits<offset_type>::max()) {
+        ctx->SetStatus(Status::CapacityError(
+            "Result might not fit in a 32bit utf8 array, convert to large_utf8"));
+        return;
+      }
+
+      KERNEL_RETURN_IF_ERROR(
+          ctx, ctx->Allocate(output_ncodeunits_max).Value(&output->buffers[2]));
+      // We could reuse the buffer if it is all ascii, benchmarking showed this not to
+      // matter
+      // output->buffers[1] = input.buffers[1];
+      KERNEL_RETURN_IF_ERROR(ctx,
+                             ctx->Allocate((input_nstrings + 1) * sizeof(offset_type))
+                                 .Value(&output->buffers[1]));
+      utf8proc_uint8_t* output_str = output->buffers[2]->mutable_data();

Review comment:
       Please use `uint8_t`




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