You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2020/06/19 02:11:27 UTC

[arrow] branch master updated: ARROW-9181: [C++] Instantiate fewer templates for cast kernels

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 28f917f  ARROW-9181: [C++] Instantiate fewer templates for cast kernels
28f917f is described below

commit 28f917ffd6b24c6c11b84efcae1b7fc023f78d16
Author: Wes McKinney <we...@apache.org>
AuthorDate: Thu Jun 18 21:10:51 2020 -0500

    ARROW-9181: [C++] Instantiate fewer templates for cast kernels
    
    I discovered this unnecessary template instantiation by looking at the symbol sizes in object files with `nm --print-size --size-sort $OBJECT_FILE`. This trims about 200K from libarrow.so in release builds on Linux.
    
    Closes #7490 from wesm/ARROW-9181
    
    Authored-by: Wes McKinney <we...@apache.org>
    Signed-off-by: Wes McKinney <we...@apache.org>
---
 .../arrow/compute/kernels/scalar_cast_boolean.cc   |  2 +-
 .../arrow/compute/kernels/scalar_cast_internal.cc  | 34 ++++++++++++++++
 .../arrow/compute/kernels/scalar_cast_internal.h   | 46 +---------------------
 .../arrow/compute/kernels/scalar_cast_nested.cc    |  8 ++--
 .../arrow/compute/kernels/scalar_cast_numeric.cc   |  6 +--
 .../arrow/compute/kernels/scalar_cast_string.cc    | 12 +++---
 .../arrow/compute/kernels/scalar_cast_temporal.cc  | 12 +++---
 7 files changed, 56 insertions(+), 64 deletions(-)

diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc b/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc
index 524866a..7b12835 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc
@@ -48,7 +48,7 @@ struct ParseBooleanString {
 
 std::vector<std::shared_ptr<CastFunction>> GetBooleanCasts() {
   auto func = std::make_shared<CastFunction>("cast_boolean", Type::BOOL);
-  AddCommonCasts<BooleanType>(boolean(), func.get());
+  AddCommonCasts(Type::BOOL, boolean(), func.get());
 
   for (const auto& ty : NumericTypes()) {
     ArrayKernelExec exec =
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc b/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc
index f9a1359..bdb27ac 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_internal.cc
@@ -66,6 +66,14 @@ void CastFromExtension(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
   out->value = casted_storage.array();
 }
 
+void CastFromNull(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  ArrayData* output = out->mutable_array();
+  std::shared_ptr<Array> nulls;
+  Status s = MakeArrayOfNull(output->type, batch.length).Value(&nulls);
+  KERNEL_RETURN_IF_ERROR(ctx, s);
+  out->value = nulls->data();
+}
+
 Result<ValueDescr> ResolveOutputFromOptions(KernelContext* ctx,
                                             const std::vector<ValueDescr>& args) {
   const CastOptions& options = checked_cast<const CastState&>(*ctx->state()).options;
@@ -112,6 +120,32 @@ void AddZeroCopyCast(Type::type in_type_id, InputType in_type, OutputType out_ty
   DCHECK_OK(func->AddKernel(in_type_id, std::move(kernel)));
 }
 
+static bool CanCastFromDictionary(Type::type type_id) {
+  return (is_primitive(type_id) || is_base_binary_like(type_id) ||
+          is_fixed_size_binary(type_id));
+}
+
+void AddCommonCasts(Type::type out_type_id, OutputType out_ty, CastFunction* func) {
+  // From null to this type
+  DCHECK_OK(func->AddKernel(Type::NA, {InputType::Array(null())}, out_ty, CastFromNull));
+
+  // From dictionary to this type
+  if (CanCastFromDictionary(out_type_id)) {
+    // Dictionary unpacking not implemented for boolean or nested types.
+    //
+    // XXX: Uses Take and does its own memory allocation for the moment. We can
+    // fix this later.
+    DCHECK_OK(func->AddKernel(
+        Type::DICTIONARY, {InputType::Array(Type::DICTIONARY)}, out_ty, UnpackDictionary,
+        NullHandling::COMPUTED_NO_PREALLOCATE, MemAllocation::NO_PREALLOCATE));
+  }
+
+  // From extension type to this type
+  DCHECK_OK(func->AddKernel(Type::EXTENSION, {InputType::Array(Type::EXTENSION)}, out_ty,
+                            CastFromExtension, NullHandling::COMPUTED_NO_PREALLOCATE,
+                            MemAllocation::NO_PREALLOCATE));
+}
+
 }  // namespace internal
 }  // namespace compute
 }  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_internal.h b/cpp/src/arrow/compute/kernels/scalar_cast_internal.h
index 4b9b2dd..a247646 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_internal.h
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_internal.h
@@ -49,16 +49,7 @@ void UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out);
 
 void OutputAllNull(KernelContext* ctx, const ExecBatch& batch, Datum* out);
 
-template <typename T>
-struct FromNullCast {
-  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
-    ArrayData* output = out->mutable_array();
-    std::shared_ptr<Array> nulls;
-    Status s = MakeArrayOfNull(output->type, batch.length).Value(&nulls);
-    KERNEL_RETURN_IF_ERROR(ctx, s);
-    out->value = nulls->data();
-  }
-};
+void CastFromNull(KernelContext* ctx, const ExecBatch& batch, Datum* out);
 
 // Adds a cast function where the functor is defined and the input and output
 // types have a type_singleton
@@ -80,40 +71,7 @@ Result<ValueDescr> ResolveOutputFromOptions(KernelContext* ctx,
 
 ARROW_EXPORT extern OutputType kOutputTargetType;
 
-template <typename T, typename Enable = void>
-struct MaybeAddFromDictionary {
-  static void Add(const OutputType& out_ty, CastFunction* func) {}
-};
-
-template <typename T>
-struct MaybeAddFromDictionary<
-    T, enable_if_t<!is_boolean_type<T>::value && !is_nested_type<T>::value &&
-                   !is_null_type<T>::value && !std::is_same<DictionaryType, T>::value>> {
-  static void Add(const OutputType& out_ty, CastFunction* func) {
-    // Dictionary unpacking not implemented for boolean or nested types.
-    //
-    // XXX: Uses Take and does its own memory allocation for the moment. We can
-    // fix this later.
-    DCHECK_OK(func->AddKernel(
-        Type::DICTIONARY, {InputType::Array(Type::DICTIONARY)}, out_ty, UnpackDictionary,
-        NullHandling::COMPUTED_NO_PREALLOCATE, MemAllocation::NO_PREALLOCATE));
-  }
-};
-
-template <typename OutType>
-void AddCommonCasts(OutputType out_ty, CastFunction* func) {
-  // From null to this type
-  DCHECK_OK(func->AddKernel(Type::NA, {InputType::Array(null())}, out_ty,
-                            FromNullCast<OutType>::Exec));
-
-  // From dictionary to this type
-  MaybeAddFromDictionary<OutType>::Add(out_ty, func);
-
-  // From extension type to this type
-  DCHECK_OK(func->AddKernel(Type::EXTENSION, {InputType::Array(Type::EXTENSION)}, out_ty,
-                            CastFromExtension, NullHandling::COMPUTED_NO_PREALLOCATE,
-                            MemAllocation::NO_PREALLOCATE));
-}
+void AddCommonCasts(Type::type out_type_id, OutputType out_ty, CastFunction* func);
 
 }  // namespace internal
 }  // namespace compute
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc b/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc
index f0366d4..85d84ef 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_nested.cc
@@ -66,22 +66,22 @@ std::vector<std::shared_ptr<CastFunction>> GetNestedCasts() {
   // We use the list<T> from the CastOptions when resolving the output type
 
   auto cast_list = std::make_shared<CastFunction>("cast_list", Type::LIST);
-  AddCommonCasts<ListType>(kOutputTargetType, cast_list.get());
+  AddCommonCasts(Type::LIST, kOutputTargetType, cast_list.get());
   AddListCast<ListType>(cast_list.get());
 
   auto cast_large_list =
       std::make_shared<CastFunction>("cast_large_list", Type::LARGE_LIST);
-  AddCommonCasts<LargeListType>(kOutputTargetType, cast_large_list.get());
+  AddCommonCasts(Type::LARGE_LIST, kOutputTargetType, cast_large_list.get());
   AddListCast<LargeListType>(cast_large_list.get());
 
   // FSL is a bit incomplete at the moment
   auto cast_fsl =
       std::make_shared<CastFunction>("cast_fixed_size_list", Type::FIXED_SIZE_LIST);
-  AddCommonCasts<FixedSizeListType>(kOutputTargetType, cast_fsl.get());
+  AddCommonCasts(Type::FIXED_SIZE_LIST, kOutputTargetType, cast_fsl.get());
 
   // So is struct
   auto cast_struct = std::make_shared<CastFunction>("cast_struct", Type::STRUCT);
-  AddCommonCasts<StructType>(kOutputTargetType, cast_struct.get());
+  AddCommonCasts(Type::STRUCT, kOutputTargetType, cast_struct.get());
 
   return {cast_list, cast_large_list, cast_fsl, cast_struct};
 }
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc b/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc
index af7632b..8f1c177 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc
@@ -476,7 +476,7 @@ struct CastFunctor<Decimal128Type, Decimal128Type> {
 template <typename OutType>
 void AddPrimitiveNumberCasts(const std::shared_ptr<DataType>& out_ty,
                              CastFunction* func) {
-  AddCommonCasts<OutType>(out_ty, func);
+  AddCommonCasts(out_ty->id(), out_ty, func);
 
   // Cast from boolean to number
   DCHECK_OK(func->AddKernel(Type::BOOL, {boolean()}, out_ty,
@@ -524,7 +524,7 @@ std::shared_ptr<CastFunction> GetCastToDecimal() {
 
   // Cast to decimal
   auto func = std::make_shared<CastFunction>("cast_decimal", Type::DECIMAL);
-  AddCommonCasts<Decimal128Type>(sig_out_ty, func.get());
+  AddCommonCasts(Type::DECIMAL, sig_out_ty, func.get());
 
   auto exec = CastFunctor<Decimal128Type, Decimal128Type>::Exec;
   // We resolve the output type of this kernel from the CastOptions
@@ -568,7 +568,7 @@ std::vector<std::shared_ptr<CastFunction>> GetNumericCasts() {
   // HalfFloat is a bit brain-damaged for now
   auto cast_half_float =
       std::make_shared<CastFunction>("cast_half_float", Type::HALF_FLOAT);
-  AddCommonCasts<HalfFloatType>(float16(), cast_half_float.get());
+  AddCommonCasts(Type::HALF_FLOAT, float16(), cast_half_float.get());
   functions.push_back(cast_half_float);
 
   functions.push_back(GetCastToFloating<FloatType>("cast_float"));
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_string.cc b/cpp/src/arrow/compute/kernels/scalar_cast_string.cc
index b842d6b..586aeaf 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_string.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_string.cc
@@ -146,19 +146,19 @@ void AddNumberToStringCasts(std::shared_ptr<DataType> out_ty, CastFunction* func
 
 std::vector<std::shared_ptr<CastFunction>> GetBinaryLikeCasts() {
   auto cast_binary = std::make_shared<CastFunction>("cast_binary", Type::BINARY);
-  AddCommonCasts<BinaryType>(binary(), cast_binary.get());
+  AddCommonCasts(Type::BINARY, binary(), cast_binary.get());
 
   auto cast_large_binary =
       std::make_shared<CastFunction>("cast_large_binary", Type::LARGE_BINARY);
-  AddCommonCasts<LargeBinaryType>(large_binary(), cast_large_binary.get());
+  AddCommonCasts(Type::LARGE_BINARY, large_binary(), cast_large_binary.get());
 
   auto cast_fsb =
       std::make_shared<CastFunction>("cast_fixed_size_binary", Type::FIXED_SIZE_BINARY);
-  AddCommonCasts<FixedSizeBinaryType>(OutputType(ResolveOutputFromOptions),
-                                      cast_fsb.get());
+  AddCommonCasts(Type::FIXED_SIZE_BINARY, OutputType(ResolveOutputFromOptions),
+                 cast_fsb.get());
 
   auto cast_string = std::make_shared<CastFunction>("cast_string", Type::STRING);
-  AddCommonCasts<StringType>(utf8(), cast_string.get());
+  AddCommonCasts(Type::STRING, utf8(), cast_string.get());
   AddNumberToStringCasts<StringType>(utf8(), cast_string.get());
   DCHECK_OK(cast_string->AddKernel(Type::BINARY, {binary()}, utf8(),
                                    CastFunctor<StringType, BinaryType>::Exec,
@@ -166,7 +166,7 @@ std::vector<std::shared_ptr<CastFunction>> GetBinaryLikeCasts() {
 
   auto cast_large_string =
       std::make_shared<CastFunction>("cast_large_string", Type::LARGE_STRING);
-  AddCommonCasts<LargeStringType>(large_utf8(), cast_large_string.get());
+  AddCommonCasts(Type::LARGE_STRING, large_utf8(), cast_large_string.get());
   AddNumberToStringCasts<LargeStringType>(large_utf8(), cast_large_string.get());
   DCHECK_OK(
       cast_large_string->AddKernel(Type::LARGE_BINARY, {large_binary()}, large_utf8(),
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc b/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc
index b2addd0..e98923d 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc
@@ -287,7 +287,7 @@ void AddCrossUnitCast(CastFunction* func) {
 std::shared_ptr<CastFunction> GetDate32Cast() {
   auto func = std::make_shared<CastFunction>("cast_date32", Type::DATE32);
   auto out_ty = date32();
-  AddCommonCasts<Date32Type>(out_ty, func.get());
+  AddCommonCasts(Type::DATE32, out_ty, func.get());
 
   // int32 -> date32
   AddZeroCopyCast(Type::INT32, int32(), date32(), func.get());
@@ -304,7 +304,7 @@ std::shared_ptr<CastFunction> GetDate32Cast() {
 std::shared_ptr<CastFunction> GetDate64Cast() {
   auto func = std::make_shared<CastFunction>("cast_date64", Type::DATE64);
   auto out_ty = date64();
-  AddCommonCasts<Date64Type>(out_ty, func.get());
+  AddCommonCasts(Type::DATE64, out_ty, func.get());
 
   // int64 -> date64
   AddZeroCopyCast(Type::INT64, int64(), date64(), func.get());
@@ -321,7 +321,7 @@ std::shared_ptr<CastFunction> GetDate64Cast() {
 
 std::shared_ptr<CastFunction> GetDurationCast() {
   auto func = std::make_shared<CastFunction>("cast_duration", Type::DURATION);
-  AddCommonCasts<DurationType>(kOutputTargetType, func.get());
+  AddCommonCasts(Type::DURATION, kOutputTargetType, func.get());
 
   auto seconds = duration(TimeUnit::SECOND);
   auto millis = duration(TimeUnit::MILLI);
@@ -339,7 +339,7 @@ std::shared_ptr<CastFunction> GetDurationCast() {
 
 std::shared_ptr<CastFunction> GetTime32Cast() {
   auto func = std::make_shared<CastFunction>("cast_time32", Type::TIME32);
-  AddCommonCasts<Date32Type>(kOutputTargetType, func.get());
+  AddCommonCasts(Type::TIME32, kOutputTargetType, func.get());
 
   // Zero copy when the unit is the same or same integer representation
   AddZeroCopyCast(Type::INT32, /*in_type=*/int32(), kOutputTargetType, func.get());
@@ -356,7 +356,7 @@ std::shared_ptr<CastFunction> GetTime32Cast() {
 
 std::shared_ptr<CastFunction> GetTime64Cast() {
   auto func = std::make_shared<CastFunction>("cast_time64", Type::TIME64);
-  AddCommonCasts<Time64Type>(kOutputTargetType, func.get());
+  AddCommonCasts(Type::TIME64, kOutputTargetType, func.get());
 
   // Zero copy when the unit is the same or same integer representation
   AddZeroCopyCast(Type::INT64, /*in_type=*/int64(), kOutputTargetType, func.get());
@@ -373,7 +373,7 @@ std::shared_ptr<CastFunction> GetTime64Cast() {
 
 std::shared_ptr<CastFunction> GetTimestampCast() {
   auto func = std::make_shared<CastFunction>("cast_timestamp", Type::TIMESTAMP);
-  AddCommonCasts<TimestampType>(kOutputTargetType, func.get());
+  AddCommonCasts(Type::TIMESTAMP, kOutputTargetType, func.get());
 
   // Same integer representation
   AddZeroCopyCast(Type::INT64, /*in_type=*/int64(), kOutputTargetType, func.get());