From 9bef0217d5e773f8ccd18827a1ac97fcce3f9807 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Mon, 23 May 2022 10:59:13 -1000 Subject: [PATCH 1/3] ARROW-16590: Consolidate utilities for dealing with row-major data into arrow/compute/row --- cpp/src/arrow/CMakeLists.txt | 12 +- cpp/src/arrow/compute/CMakeLists.txt | 1 + cpp/src/arrow/compute/api.h | 6 + cpp/src/arrow/compute/api_aggregate.h | 93 +- cpp/src/arrow/compute/exec/aggregate_node.cc | 24 +- cpp/src/arrow/compute/exec/hash_join.cc | 3 +- cpp/src/arrow/compute/exec/key_encode.h | 502 ------- cpp/src/arrow/compute/exec/key_hash.cc | 10 +- cpp/src/arrow/compute/exec/key_hash.h | 10 +- cpp/src/arrow/compute/exec/schema_util.h | 4 +- .../arrow/compute/kernels/hash_aggregate.cc | 542 +------- .../compute/kernels/hash_aggregate_test.cc | 56 +- cpp/src/arrow/compute/light_array.h | 14 + cpp/src/arrow/compute/row/CMakeLists.txt | 21 + .../compare_internal.cc} | 50 +- .../key_compare.h => row/compare_internal.h} | 94 +- .../compare_internal_avx2.cc} | 33 +- .../key_encode.cc => row/encode_internal.cc} | 1151 ++++++----------- cpp/src/arrow/compute/row/encode_internal.h | 323 +++++ .../encode_internal_avx2.cc} | 45 +- cpp/src/arrow/compute/row/grouper.cc | 590 +++++++++ cpp/src/arrow/compute/row/grouper.h | 110 ++ cpp/src/arrow/compute/row/row_internal.cc | 409 ++++++ cpp/src/arrow/compute/row/row_internal.h | 244 ++++ cpp/src/arrow/dataset/partition.cc | 9 +- 25 files changed, 2277 insertions(+), 2079 deletions(-) delete mode 100644 cpp/src/arrow/compute/exec/key_encode.h create mode 100644 cpp/src/arrow/compute/row/CMakeLists.txt rename cpp/src/arrow/compute/{exec/key_compare.cc => row/compare_internal.cc} (90%) rename cpp/src/arrow/compute/{exec/key_compare.h => row/compare_internal.h} (52%) rename cpp/src/arrow/compute/{exec/key_compare_avx2.cc => row/compare_internal_avx2.cc} (96%) rename cpp/src/arrow/compute/{exec/key_encode.cc => row/encode_internal.cc} (56%) create mode 100644 cpp/src/arrow/compute/row/encode_internal.h rename cpp/src/arrow/compute/{exec/key_encode_avx2.cc => row/encode_internal_avx2.cc} (83%) create mode 100644 cpp/src/arrow/compute/row/grouper.cc create mode 100644 cpp/src/arrow/compute/row/grouper.h create mode 100644 cpp/src/arrow/compute/row/row_internal.cc create mode 100644 cpp/src/arrow/compute/row/row_internal.h diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index ec6cada1cda..d8e662b63f1 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -389,8 +389,6 @@ if(ARROW_COMPUTE) compute/exec/hash_join_dict.cc compute/exec/hash_join_node.cc compute/exec/ir_consumer.cc - compute/exec/key_compare.cc - compute/exec/key_encode.cc compute/exec/key_hash.cc compute/exec/key_map.cc compute/exec/order_by_impl.cc @@ -441,17 +439,21 @@ if(ARROW_COMPUTE) compute/kernels/vector_nested.cc compute/kernels/vector_replace.cc compute/kernels/vector_selection.cc - compute/kernels/vector_sort.cc) + compute/kernels/vector_sort.cc + compute/row/encode_internal.cc + compute/row/compare_internal.cc + compute/row/grouper.cc + compute/row/row_internal.cc) append_avx2_src(compute/kernels/aggregate_basic_avx2.cc) append_avx512_src(compute/kernels/aggregate_basic_avx512.cc) append_avx2_src(compute/exec/bloom_filter_avx2.cc) - append_avx2_src(compute/exec/key_compare_avx2.cc) - append_avx2_src(compute/exec/key_encode_avx2.cc) append_avx2_src(compute/exec/key_hash_avx2.cc) append_avx2_src(compute/exec/key_map_avx2.cc) append_avx2_src(compute/exec/util_avx2.cc) + append_avx2_src(compute/row/compare_internal_avx2.cc) + append_avx2_src(compute/row/encode_internal_avx2.cc) list(APPEND ARROW_TESTING_SRCS compute/exec/test_util.cc) endif() diff --git a/cpp/src/arrow/compute/CMakeLists.txt b/cpp/src/arrow/compute/CMakeLists.txt index 27e693d9a34..91fa796f6d4 100644 --- a/cpp/src/arrow/compute/CMakeLists.txt +++ b/cpp/src/arrow/compute/CMakeLists.txt @@ -71,3 +71,4 @@ add_arrow_benchmark(function_benchmark PREFIX "arrow-compute") add_subdirectory(kernels) add_subdirectory(exec) +add_subdirectory(row) diff --git a/cpp/src/arrow/compute/api.h b/cpp/src/arrow/compute/api.h index c8e0c2ee234..80582e47b74 100644 --- a/cpp/src/arrow/compute/api.h +++ b/cpp/src/arrow/compute/api.h @@ -46,3 +46,9 @@ /// @} #include "arrow/compute/exec/options.h" // IWYU pragma: export + +/// \defgroup execnode-row Utilities for working with data in a row-major format +/// @{ +/// @} + +#include "arrow/compute/row/grouper.h" // IWYU pragma: export diff --git a/cpp/src/arrow/compute/api_aggregate.h b/cpp/src/arrow/compute/api_aggregate.h index 1255a57ff1d..930c5d95230 100644 --- a/cpp/src/arrow/compute/api_aggregate.h +++ b/cpp/src/arrow/compute/api_aggregate.h @@ -395,84 +395,6 @@ Result Index(const Datum& value, const IndexOptions& options, namespace internal { -/// Internal use only: streaming group identifier. -/// Consumes batches of keys and yields batches of the group ids. -class ARROW_EXPORT Grouper { - public: - virtual ~Grouper() = default; - - /// Construct a Grouper which receives the specified key types - static Result> Make(const std::vector& descrs, - ExecContext* ctx = default_exec_context()); - - /// Consume a batch of keys, producing the corresponding group ids as an integer array. - /// Currently only uint32 indices will be produced, eventually the bit width will only - /// be as wide as necessary. - virtual Result Consume(const ExecBatch& batch) = 0; - - /// Get current unique keys. May be called multiple times. - virtual Result GetUniques() = 0; - - /// Get the current number of groups. - virtual uint32_t num_groups() const = 0; - - /// \brief Assemble lists of indices of identical elements. - /// - /// \param[in] ids An unsigned, all-valid integral array which will be - /// used as grouping criteria. - /// \param[in] num_groups An upper bound for the elements of ids - /// \return A num_groups-long ListArray where the slot at i contains a - /// list of indices where i appears in ids. - /// - /// MakeGroupings([ - /// 2, - /// 2, - /// 5, - /// 5, - /// 2, - /// 3 - /// ], 8) == [ - /// [], - /// [], - /// [0, 1, 4], - /// [5], - /// [], - /// [2, 3], - /// [], - /// [] - /// ] - static Result> MakeGroupings( - const UInt32Array& ids, uint32_t num_groups, - ExecContext* ctx = default_exec_context()); - - /// \brief Produce a ListArray whose slots are selections of `array` which correspond to - /// the provided groupings. - /// - /// For example, - /// ApplyGroupings([ - /// [], - /// [], - /// [0, 1, 4], - /// [5], - /// [], - /// [2, 3], - /// [], - /// [] - /// ], [2, 2, 5, 5, 2, 3]) == [ - /// [], - /// [], - /// [2, 2, 2], - /// [3], - /// [], - /// [5, 5], - /// [], - /// [] - /// ] - static Result> ApplyGroupings( - const ListArray& groupings, const Array& array, - ExecContext* ctx = default_exec_context()); -}; - /// \brief Configure a grouped aggregation struct ARROW_EXPORT Aggregate { /// the name of the aggregation function @@ -482,6 +404,21 @@ struct ARROW_EXPORT Aggregate { const FunctionOptions* options; }; +Result> GetKernels( + ExecContext* ctx, const std::vector& aggregates, + const std::vector& in_descrs); + +Result>> InitKernels( + const std::vector& kernels, ExecContext* ctx, + const std::vector& aggregates, + const std::vector& in_descrs); + +Result ResolveKernels( + const std::vector& aggregates, + const std::vector& kernels, + const std::vector>& states, ExecContext* ctx, + const std::vector& descrs); + /// Internal use only: helper function for testing HashAggregateKernels. /// This will be replaced by streaming execution operators. ARROW_EXPORT diff --git a/cpp/src/arrow/compute/exec/aggregate_node.cc b/cpp/src/arrow/compute/exec/aggregate_node.cc index f6813ecb682..edc600fbfc1 100644 --- a/cpp/src/arrow/compute/exec/aggregate_node.cc +++ b/cpp/src/arrow/compute/exec/aggregate_node.cc @@ -26,6 +26,7 @@ #include "arrow/compute/exec/util.h" #include "arrow/compute/exec_internal.h" #include "arrow/compute/registry.h" +#include "arrow/compute/row/grouper.h" #include "arrow/datum.h" #include "arrow/result.h" #include "arrow/util/checked_cast.h" @@ -39,25 +40,6 @@ using internal::checked_cast; namespace compute { -namespace internal { - -Result> GetKernels( - ExecContext* ctx, const std::vector& aggregates, - const std::vector& in_descrs); - -Result>> InitKernels( - const std::vector& kernels, ExecContext* ctx, - const std::vector& aggregates, - const std::vector& in_descrs); - -Result ResolveKernels( - const std::vector& aggregates, - const std::vector& kernels, - const std::vector>& states, ExecContext* ctx, - const std::vector& descrs); - -} // namespace internal - namespace { void AggregatesToString( @@ -647,7 +629,7 @@ class GroupByNode : public ExecNode { private: struct ThreadLocalState { - std::unique_ptr grouper; + std::unique_ptr grouper; std::vector> agg_states; }; @@ -670,7 +652,7 @@ class GroupByNode : public ExecNode { } // Construct grouper - ARROW_ASSIGN_OR_RAISE(state->grouper, internal::Grouper::Make(key_descrs, ctx_)); + ARROW_ASSIGN_OR_RAISE(state->grouper, Grouper::Make(key_descrs, ctx_)); // Build vector of aggregate source field data types std::vector agg_src_descrs(agg_kernels_.size()); diff --git a/cpp/src/arrow/compute/exec/hash_join.cc b/cpp/src/arrow/compute/exec/hash_join.cc index 15a006c81d5..5e46deae2fb 100644 --- a/cpp/src/arrow/compute/exec/hash_join.cc +++ b/cpp/src/arrow/compute/exec/hash_join.cc @@ -29,12 +29,11 @@ #include "arrow/compute/exec/key_hash.h" #include "arrow/compute/exec/task_util.h" #include "arrow/compute/kernels/row_encoder.h" +#include "arrow/compute/row/encode_internal.h" namespace arrow { namespace compute { -using internal::RowEncoder; - class HashJoinBasicImpl : public HashJoinImpl { private: struct ThreadLocalState; diff --git a/cpp/src/arrow/compute/exec/key_encode.h b/cpp/src/arrow/compute/exec/key_encode.h deleted file mode 100644 index 58d8fb233f0..00000000000 --- a/cpp/src/arrow/compute/exec/key_encode.h +++ /dev/null @@ -1,502 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#pragma once - -#include -#include -#include - -#include "arrow/array/data.h" -#include "arrow/compute/exec.h" -#include "arrow/compute/exec/util.h" -#include "arrow/compute/light_array.h" -#include "arrow/memory_pool.h" -#include "arrow/result.h" -#include "arrow/status.h" -#include "arrow/util/bit_util.h" - -namespace arrow { -namespace compute { - -/// Converts between key representation as a collection of arrays for -/// individual columns and another representation as a single array of rows -/// combining data from all columns into one value. -/// This conversion is reversible. -/// Row-oriented storage is beneficial when there is a need for random access -/// of individual rows and at the same time all included columns are likely to -/// be accessed together, as in the case of hash table key. -class KeyEncoder { - public: - struct KeyEncoderContext { - bool has_avx2() const { - return (hardware_flags & arrow::internal::CpuInfo::AVX2) > 0; - } - int64_t hardware_flags; - util::TempVectorStack* stack; - }; - - /// Description of a storage format for rows produced by encoder. - struct KeyRowMetadata { - /// Is row a varying-length binary, using offsets array to find a beginning of a row, - /// or is it a fixed-length binary. - bool is_fixed_length; - - /// For a fixed-length binary row, common size of rows in bytes, - /// rounded up to the multiple of alignment. - /// - /// For a varying-length binary, size of all encoded fixed-length key columns, - /// including lengths of varying-length columns, rounded up to the multiple of string - /// alignment. - uint32_t fixed_length; - - /// Offset within a row to the array of 32-bit offsets within a row of - /// ends of varbinary fields. - /// Used only when the row is not fixed-length, zero for fixed-length row. - /// There are N elements for N varbinary fields. - /// Each element is the offset within a row of the first byte after - /// the corresponding varbinary field bytes in that row. - /// If varbinary fields begin at aligned addresses, than the end of the previous - /// varbinary field needs to be rounded up according to the specified alignment - /// to obtain the beginning of the next varbinary field. - /// The first varbinary field starts at offset specified by fixed_length, - /// which should already be aligned. - uint32_t varbinary_end_array_offset; - - /// Fixed number of bytes per row that are used to encode null masks. - /// Null masks indicate for a single row which of its key columns are null. - /// Nth bit in the sequence of bytes assigned to a row represents null - /// information for Nth field according to the order in which they are encoded. - int null_masks_bytes_per_row; - - /// Power of 2. Every row will start at the offset aligned to that number of bytes. - int row_alignment; - - /// Power of 2. Must be no greater than row alignment. - /// Every non-power-of-2 binary field and every varbinary field bytes - /// will start aligned to that number of bytes. - int string_alignment; - - /// Metadata of encoded columns in their original order. - std::vector column_metadatas; - - /// Order in which fields are encoded. - std::vector column_order; - - /// Offsets within a row to fields in their encoding order. - std::vector column_offsets; - - /// Rounding up offset to the nearest multiple of alignment value. - /// Alignment must be a power of 2. - static inline uint32_t padding_for_alignment(uint32_t offset, - int required_alignment) { - ARROW_DCHECK(ARROW_POPCOUNT64(required_alignment) == 1); - return static_cast((-static_cast(offset)) & - (required_alignment - 1)); - } - - /// Rounding up offset to the beginning of next column, - /// chosing required alignment based on the data type of that column. - static inline uint32_t padding_for_alignment(uint32_t offset, int string_alignment, - const KeyColumnMetadata& col_metadata) { - if (!col_metadata.is_fixed_length || - ARROW_POPCOUNT64(col_metadata.fixed_length) <= 1) { - return 0; - } else { - return padding_for_alignment(offset, string_alignment); - } - } - - /// Returns an array of offsets within a row of ends of varbinary fields. - inline const uint32_t* varbinary_end_array(const uint8_t* row) const { - ARROW_DCHECK(!is_fixed_length); - return reinterpret_cast(row + varbinary_end_array_offset); - } - inline uint32_t* varbinary_end_array(uint8_t* row) const { - ARROW_DCHECK(!is_fixed_length); - return reinterpret_cast(row + varbinary_end_array_offset); - } - - /// Returns the offset within the row and length of the first varbinary field. - inline void first_varbinary_offset_and_length(const uint8_t* row, uint32_t* offset, - uint32_t* length) const { - ARROW_DCHECK(!is_fixed_length); - *offset = fixed_length; - *length = varbinary_end_array(row)[0] - fixed_length; - } - - /// Returns the offset within the row and length of the second and further varbinary - /// fields. - inline void nth_varbinary_offset_and_length(const uint8_t* row, int varbinary_id, - uint32_t* out_offset, - uint32_t* out_length) const { - ARROW_DCHECK(!is_fixed_length); - ARROW_DCHECK(varbinary_id > 0); - const uint32_t* varbinary_end = varbinary_end_array(row); - uint32_t offset = varbinary_end[varbinary_id - 1]; - offset += padding_for_alignment(offset, string_alignment); - *out_offset = offset; - *out_length = varbinary_end[varbinary_id] - offset; - } - - uint32_t encoded_field_order(uint32_t icol) const { return column_order[icol]; } - - uint32_t encoded_field_offset(uint32_t icol) const { return column_offsets[icol]; } - - uint32_t num_cols() const { return static_cast(column_metadatas.size()); } - - uint32_t num_varbinary_cols() const; - - void FromColumnMetadataVector(const std::vector& cols, - int in_row_alignment, int in_string_alignment); - - bool is_compatible(const KeyRowMetadata& other) const; - }; - - class KeyRowArray { - public: - KeyRowArray(); - Status Init(MemoryPool* pool, const KeyRowMetadata& metadata); - void Clean(); - Status AppendEmpty(uint32_t num_rows_to_append, uint32_t num_extra_bytes_to_append); - Status AppendSelectionFrom(const KeyRowArray& from, uint32_t num_rows_to_append, - const uint16_t* source_row_ids); - const KeyRowMetadata& metadata() const { return metadata_; } - int64_t length() const { return num_rows_; } - const uint8_t* data(int i) const { - ARROW_DCHECK(i >= 0 && i <= max_buffers_); - return buffers_[i]; - } - uint8_t* mutable_data(int i) { - ARROW_DCHECK(i >= 0 && i <= max_buffers_); - return mutable_buffers_[i]; - } - const uint32_t* offsets() const { return reinterpret_cast(data(1)); } - uint32_t* mutable_offsets() { return reinterpret_cast(mutable_data(1)); } - const uint8_t* null_masks() const { return null_masks_->data(); } - uint8_t* null_masks() { return null_masks_->mutable_data(); } - - bool has_any_nulls(const KeyEncoderContext* ctx) const; - - private: - Status ResizeFixedLengthBuffers(int64_t num_extra_rows); - Status ResizeOptionalVaryingLengthBuffer(int64_t num_extra_bytes); - - int64_t size_null_masks(int64_t num_rows); - int64_t size_offsets(int64_t num_rows); - int64_t size_rows_fixed_length(int64_t num_rows); - int64_t size_rows_varying_length(int64_t num_bytes); - void update_buffer_pointers(); - - static constexpr int64_t padding_for_vectors = 64; - MemoryPool* pool_; - KeyRowMetadata metadata_; - /// Buffers can only expand during lifetime and never shrink. - std::unique_ptr null_masks_; - std::unique_ptr offsets_; - std::unique_ptr rows_; - static constexpr int max_buffers_ = 3; - const uint8_t* buffers_[max_buffers_]; - uint8_t* mutable_buffers_[max_buffers_]; - int64_t num_rows_; - int64_t rows_capacity_; - int64_t bytes_capacity_; - - // Mutable to allow lazy evaluation - mutable int64_t num_rows_for_has_any_nulls_; - mutable bool has_any_nulls_; - }; - - void Init(const std::vector& cols, KeyEncoderContext* ctx, - int row_alignment, int string_alignment); - - const KeyRowMetadata& row_metadata() { return row_metadata_; } - - void PrepareEncodeSelected(int64_t start_row, int64_t num_rows, - const std::vector& cols); - Status EncodeSelected(KeyRowArray* rows, uint32_t num_selected, - const uint16_t* selection); - - /// Decode a window of row oriented data into a corresponding - /// window of column oriented storage. - /// The output buffers need to be correctly allocated and sized before - /// calling each method. - /// For that reason decoding is split into two functions. - /// The output of the first one, that processes everything except for - /// varying length buffers, can be used to find out required varying - /// length buffers sizes. - void DecodeFixedLengthBuffers(int64_t start_row_input, int64_t start_row_output, - int64_t num_rows, const KeyRowArray& rows, - std::vector* cols); - - void DecodeVaryingLengthBuffers(int64_t start_row_input, int64_t start_row_output, - int64_t num_rows, const KeyRowArray& rows, - std::vector* cols); - - const std::vector& GetBatchColumns() const { return batch_all_cols_; } - - private: - /// Prepare column array vectors. - /// Output column arrays represent a range of input column arrays - /// specified by starting row and number of rows. - /// Three vectors are generated: - /// - all columns - /// - fixed-length columns only - /// - varying-length columns only - void PrepareKeyColumnArrays(int64_t start_row, int64_t num_rows, - const std::vector& cols_in); - - class TransformBoolean { - public: - static KeyColumnArray ArrayReplace(const KeyColumnArray& column, - const KeyColumnArray& temp); - static void PostDecode(const KeyColumnArray& input, KeyColumnArray* output, - KeyEncoderContext* ctx); - }; - - class EncoderInteger { - public: - static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col, - KeyEncoderContext* ctx, KeyColumnArray* temp); - static bool UsesTransform(const KeyColumnArray& column); - static KeyColumnArray ArrayReplace(const KeyColumnArray& column, - const KeyColumnArray& temp); - static void PostDecode(const KeyColumnArray& input, KeyColumnArray* output, - KeyEncoderContext* ctx); - - private: - static bool IsBoolean(const KeyColumnMetadata& metadata); - }; - - class EncoderBinary { - public: - static void EncodeSelected(uint32_t offset_within_row, KeyRowArray* rows, - const KeyColumnArray& col, uint32_t num_selected, - const uint16_t* selection); - static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col, - KeyEncoderContext* ctx, KeyColumnArray* temp); - static bool IsInteger(const KeyColumnMetadata& metadata); - - private: - template - static void EncodeSelectedImp(uint32_t offset_within_row, KeyRowArray* rows, - const KeyColumnArray& col, uint32_t num_selected, - const uint16_t* selection, COPY_FN copy_fn, - SET_NULL_FN set_null_fn); - - template - static inline void DecodeHelper(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray* rows_const, - KeyRowArray* rows_mutable_maybe_null, - const KeyColumnArray* col_const, - KeyColumnArray* col_mutable_maybe_null, - COPY_FN copy_fn); - template - static void DecodeImp(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, const KeyRowArray& rows, - KeyColumnArray* col); -#if defined(ARROW_HAVE_AVX2) - static void DecodeHelper_avx2(bool is_row_fixed_length, uint32_t start_row, - uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col); - template - static void DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, const KeyRowArray& rows, - KeyColumnArray* col); -#endif - }; - - class EncoderBinaryPair { - public: - static bool CanProcessPair(const KeyColumnMetadata& col1, - const KeyColumnMetadata& col2) { - return EncoderBinary::IsInteger(col1) && EncoderBinary::IsInteger(col2); - } - static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col1, - KeyColumnArray* col2, KeyEncoderContext* ctx, - KeyColumnArray* temp1, KeyColumnArray* temp2); - - private: - template - static void DecodeImp(uint32_t num_rows_to_skip, uint32_t start_row, - uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col1, - KeyColumnArray* col2); -#if defined(ARROW_HAVE_AVX2) - static uint32_t DecodeHelper_avx2(bool is_row_fixed_length, uint32_t col_width, - uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, const KeyRowArray& rows, - KeyColumnArray* col1, KeyColumnArray* col2); - template - static uint32_t DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, const KeyRowArray& rows, - KeyColumnArray* col1, KeyColumnArray* col2); -#endif - }; - - class EncoderOffsets { - public: - static void GetRowOffsetsSelected(KeyRowArray* rows, - const std::vector& cols, - uint32_t num_selected, const uint16_t* selection); - static void EncodeSelected(KeyRowArray* rows, const std::vector& cols, - uint32_t num_selected, const uint16_t* selection); - - static void Decode(uint32_t start_row, uint32_t num_rows, const KeyRowArray& rows, - std::vector* varbinary_cols, - const std::vector& varbinary_cols_base_offset, - KeyEncoderContext* ctx); - - private: - template - static void EncodeSelectedImp(uint32_t ivarbinary, KeyRowArray* rows, - const std::vector& cols, - uint32_t num_selected, const uint16_t* selection); - }; - - class EncoderVarBinary { - public: - static void EncodeSelected(uint32_t ivarbinary, KeyRowArray* rows, - const KeyColumnArray& cols, uint32_t num_selected, - const uint16_t* selection); - - static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t varbinary_col_id, - const KeyRowArray& rows, KeyColumnArray* col, - KeyEncoderContext* ctx); - - private: - template - static inline void DecodeHelper(uint32_t start_row, uint32_t num_rows, - uint32_t varbinary_col_id, - const KeyRowArray* rows_const, - KeyRowArray* rows_mutable_maybe_null, - const KeyColumnArray* col_const, - KeyColumnArray* col_mutable_maybe_null, - COPY_FN copy_fn); - template - static void DecodeImp(uint32_t start_row, uint32_t num_rows, - uint32_t varbinary_col_id, const KeyRowArray& rows, - KeyColumnArray* col); -#if defined(ARROW_HAVE_AVX2) - static void DecodeHelper_avx2(uint32_t start_row, uint32_t num_rows, - uint32_t varbinary_col_id, const KeyRowArray& rows, - KeyColumnArray* col); - template - static void DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, - uint32_t varbinary_col_id, const KeyRowArray& rows, - KeyColumnArray* col); -#endif - }; - - class EncoderNulls { - public: - static void EncodeSelected(KeyRowArray* rows, const std::vector& cols, - uint32_t num_selected, const uint16_t* selection); - - static void Decode(uint32_t start_row, uint32_t num_rows, const KeyRowArray& rows, - std::vector* cols); - }; - - KeyEncoderContext* ctx_; - - // Data initialized once, based on data types of key columns - KeyRowMetadata row_metadata_; - - // Data initialized for each input batch. - // All elements are ordered according to the order of encoded fields in a row. - std::vector batch_all_cols_; - std::vector batch_varbinary_cols_; - std::vector batch_varbinary_cols_base_offsets_; -}; - -template -inline void KeyEncoder::EncoderBinary::DecodeHelper( - uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray* rows_const, KeyRowArray* rows_mutable_maybe_null, - const KeyColumnArray* col_const, KeyColumnArray* col_mutable_maybe_null, - COPY_FN copy_fn) { - ARROW_DCHECK(col_const && col_const->metadata().is_fixed_length); - uint32_t col_width = col_const->metadata().fixed_length; - - if (is_row_fixed_length) { - uint32_t row_width = rows_const->metadata().fixed_length; - for (uint32_t i = 0; i < num_rows; ++i) { - const uint8_t* src; - uint8_t* dst; - src = rows_const->data(1) + row_width * (start_row + i) + offset_within_row; - dst = col_mutable_maybe_null->mutable_data(1) + col_width * i; - copy_fn(dst, src, col_width); - } - } else { - const uint32_t* row_offsets = rows_const->offsets(); - for (uint32_t i = 0; i < num_rows; ++i) { - const uint8_t* src; - uint8_t* dst; - src = rows_const->data(2) + row_offsets[start_row + i] + offset_within_row; - dst = col_mutable_maybe_null->mutable_data(1) + col_width * i; - copy_fn(dst, src, col_width); - } - } -} - -template -inline void KeyEncoder::EncoderVarBinary::DecodeHelper( - uint32_t start_row, uint32_t num_rows, uint32_t varbinary_col_id, - const KeyRowArray* rows_const, KeyRowArray* rows_mutable_maybe_null, - const KeyColumnArray* col_const, KeyColumnArray* col_mutable_maybe_null, - COPY_FN copy_fn) { - // Column and rows need to be varying length - ARROW_DCHECK(!rows_const->metadata().is_fixed_length && - !col_const->metadata().is_fixed_length); - - const uint32_t* row_offsets_for_batch = rows_const->offsets() + start_row; - const uint32_t* col_offsets = col_const->offsets(); - - uint32_t col_offset_next = col_offsets[0]; - for (uint32_t i = 0; i < num_rows; ++i) { - uint32_t col_offset = col_offset_next; - col_offset_next = col_offsets[i + 1]; - - uint32_t row_offset = row_offsets_for_batch[i]; - const uint8_t* row = rows_const->data(2) + row_offset; - - uint32_t offset_within_row; - uint32_t length; - if (first_varbinary_col) { - rows_const->metadata().first_varbinary_offset_and_length(row, &offset_within_row, - &length); - } else { - rows_const->metadata().nth_varbinary_offset_and_length(row, varbinary_col_id, - &offset_within_row, &length); - } - - row_offset += offset_within_row; - - const uint8_t* src; - uint8_t* dst; - src = rows_const->data(2) + row_offset; - dst = col_mutable_maybe_null->mutable_data(2) + col_offset; - copy_fn(dst, src, length); - } -} - -} // namespace compute -} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/key_hash.cc b/cpp/src/arrow/compute/exec/key_hash.cc index e81ed64b6f2..5a5d524c404 100644 --- a/cpp/src/arrow/compute/exec/key_hash.cc +++ b/cpp/src/arrow/compute/exec/key_hash.cc @@ -22,7 +22,7 @@ #include #include -#include "arrow/compute/exec/key_encode.h" +#include "arrow/compute/light_array.h" #include "arrow/util/bit_util.h" #include "arrow/util/ubsan.h" @@ -377,7 +377,7 @@ void Hashing32::HashFixed(int64_t hardware_flags, bool combine_hashes, uint32_t } void Hashing32::HashMultiColumn(const std::vector& cols, - KeyEncoder::KeyEncoderContext* ctx, uint32_t* hashes) { + LightContext* ctx, uint32_t* hashes) { uint32_t num_rows = static_cast(cols[0].length()); constexpr uint32_t max_batch_size = util::MiniBatch::kMiniBatchLength; @@ -463,7 +463,7 @@ Status Hashing32::HashBatch(const ExecBatch& key_batch, uint32_t* hashes, std::vector column_arrays; RETURN_NOT_OK(ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays)); - KeyEncoder::KeyEncoderContext ctx; + LightContext ctx; ctx.hardware_flags = hardware_flags; ctx.stack = temp_stack; HashMultiColumn(column_arrays, &ctx, hashes); @@ -814,7 +814,7 @@ void Hashing64::HashFixed(bool combine_hashes, uint32_t num_rows, uint64_t lengt } void Hashing64::HashMultiColumn(const std::vector& cols, - KeyEncoder::KeyEncoderContext* ctx, uint64_t* hashes) { + LightContext* ctx, uint64_t* hashes) { uint32_t num_rows = static_cast(cols[0].length()); constexpr uint32_t max_batch_size = util::MiniBatch::kMiniBatchLength; @@ -895,7 +895,7 @@ Status Hashing64::HashBatch(const ExecBatch& key_batch, uint64_t* hashes, std::vector column_arrays; RETURN_NOT_OK(ColumnArraysFromExecBatch(key_batch, offset, length, &column_arrays)); - KeyEncoder::KeyEncoderContext ctx; + LightContext ctx; ctx.hardware_flags = hardware_flags; ctx.stack = temp_stack; HashMultiColumn(column_arrays, &ctx, hashes); diff --git a/cpp/src/arrow/compute/exec/key_hash.h b/cpp/src/arrow/compute/exec/key_hash.h index 05f39bb729a..f8af7988387 100644 --- a/cpp/src/arrow/compute/exec/key_hash.h +++ b/cpp/src/arrow/compute/exec/key_hash.h @@ -23,8 +23,8 @@ #include -#include "arrow/compute/exec/key_encode.h" #include "arrow/compute/exec/util.h" +#include "arrow/compute/light_array.h" namespace arrow { namespace compute { @@ -45,8 +45,8 @@ class ARROW_EXPORT Hashing32 { friend void TestBloomSmall(BloomFilterBuildStrategy, int64_t, int, bool, bool); public: - static void HashMultiColumn(const std::vector& cols, - KeyEncoder::KeyEncoderContext* ctx, uint32_t* out_hash); + static void HashMultiColumn(const std::vector& cols, LightContext* ctx, + uint32_t* out_hash); static Status HashBatch(const ExecBatch& key_batch, uint32_t* hashes, int64_t hardware_flags, util::TempVectorStack* temp_stack, @@ -157,8 +157,8 @@ class ARROW_EXPORT Hashing64 { friend void TestBloomSmall(BloomFilterBuildStrategy, int64_t, int, bool, bool); public: - static void HashMultiColumn(const std::vector& cols, - KeyEncoder::KeyEncoderContext* ctx, uint64_t* hashes); + static void HashMultiColumn(const std::vector& cols, LightContext* ctx, + uint64_t* hashes); static Status HashBatch(const ExecBatch& key_batch, uint64_t* hashes, int64_t hardware_flags, util::TempVectorStack* temp_stack, diff --git a/cpp/src/arrow/compute/exec/schema_util.h b/cpp/src/arrow/compute/exec/schema_util.h index 4e307e23807..91b7e6cfc6e 100644 --- a/cpp/src/arrow/compute/exec/schema_util.h +++ b/cpp/src/arrow/compute/exec/schema_util.h @@ -22,8 +22,8 @@ #include #include -#include "arrow/compute/exec/key_encode.h" // for KeyColumnMetadata -#include "arrow/type.h" // for DataType, FieldRef, Field and Schema +#include "arrow/compute/light_array.h" // for KeyColumnMetadata +#include "arrow/type.h" // for DataType, FieldRef, Field and Schema #include "arrow/util/mutex.h" namespace arrow { diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate.cc b/cpp/src/arrow/compute/kernels/hash_aggregate.cc index d9ffcda5962..bceb972c15b 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate.cc @@ -29,8 +29,6 @@ #include "arrow/buffer_builder.h" #include "arrow/compute/api_aggregate.h" #include "arrow/compute/api_vector.h" -#include "arrow/compute/exec/key_compare.h" -#include "arrow/compute/exec/key_encode.h" #include "arrow/compute/exec/key_hash.h" #include "arrow/compute/exec/key_map.h" #include "arrow/compute/exec/util.h" @@ -41,6 +39,7 @@ #include "arrow/compute/kernels/common.h" #include "arrow/compute/kernels/row_encoder.h" #include "arrow/compute/kernels/util_internal.h" +#include "arrow/compute/row/grouper.h" #include "arrow/record_batch.h" #include "arrow/stl_allocator.h" #include "arrow/util/bit_run_reader.h" @@ -65,485 +64,6 @@ namespace compute { namespace internal { namespace { -struct GrouperImpl : Grouper { - static Result> Make(const std::vector& keys, - ExecContext* ctx) { - auto impl = ::arrow::internal::make_unique(); - - impl->encoders_.resize(keys.size()); - impl->ctx_ = ctx; - - for (size_t i = 0; i < keys.size(); ++i) { - const auto& key = keys[i].type; - - if (key->id() == Type::BOOL) { - impl->encoders_[i] = ::arrow::internal::make_unique(); - continue; - } - - if (key->id() == Type::DICTIONARY) { - impl->encoders_[i] = - ::arrow::internal::make_unique(key, ctx->memory_pool()); - continue; - } - - if (is_fixed_width(key->id())) { - impl->encoders_[i] = ::arrow::internal::make_unique(key); - continue; - } - - if (is_binary_like(key->id())) { - impl->encoders_[i] = - ::arrow::internal::make_unique>(key); - continue; - } - - if (is_large_binary_like(key->id())) { - impl->encoders_[i] = - ::arrow::internal::make_unique>(key); - continue; - } - - if (key->id() == Type::NA) { - impl->encoders_[i] = ::arrow::internal::make_unique(); - continue; - } - - return Status::NotImplemented("Keys of type ", *key); - } - - return std::move(impl); - } - - Result Consume(const ExecBatch& batch) override { - std::vector offsets_batch(batch.length + 1); - for (int i = 0; i < batch.num_values(); ++i) { - encoders_[i]->AddLength(batch[i], batch.length, offsets_batch.data()); - } - - int32_t total_length = 0; - for (int64_t i = 0; i < batch.length; ++i) { - auto total_length_before = total_length; - total_length += offsets_batch[i]; - offsets_batch[i] = total_length_before; - } - offsets_batch[batch.length] = total_length; - - std::vector key_bytes_batch(total_length); - std::vector key_buf_ptrs(batch.length); - for (int64_t i = 0; i < batch.length; ++i) { - key_buf_ptrs[i] = key_bytes_batch.data() + offsets_batch[i]; - } - - for (int i = 0; i < batch.num_values(); ++i) { - RETURN_NOT_OK(encoders_[i]->Encode(batch[i], batch.length, key_buf_ptrs.data())); - } - - TypedBufferBuilder group_ids_batch(ctx_->memory_pool()); - RETURN_NOT_OK(group_ids_batch.Resize(batch.length)); - - for (int64_t i = 0; i < batch.length; ++i) { - int32_t key_length = offsets_batch[i + 1] - offsets_batch[i]; - std::string key( - reinterpret_cast(key_bytes_batch.data() + offsets_batch[i]), - key_length); - - auto it_success = map_.emplace(key, num_groups_); - auto group_id = it_success.first->second; - - if (it_success.second) { - // new key; update offsets and key_bytes - ++num_groups_; - // Skip if there are no keys - if (key_length > 0) { - auto next_key_offset = static_cast(key_bytes_.size()); - key_bytes_.resize(next_key_offset + key_length); - offsets_.push_back(next_key_offset + key_length); - memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); - } - } - - group_ids_batch.UnsafeAppend(group_id); - } - - ARROW_ASSIGN_OR_RAISE(auto group_ids, group_ids_batch.Finish()); - return Datum(UInt32Array(batch.length, std::move(group_ids))); - } - - uint32_t num_groups() const override { return num_groups_; } - - Result GetUniques() override { - ExecBatch out({}, num_groups_); - - std::vector key_buf_ptrs(num_groups_); - for (int64_t i = 0; i < num_groups_; ++i) { - key_buf_ptrs[i] = key_bytes_.data() + offsets_[i]; - } - - out.values.resize(encoders_.size()); - for (size_t i = 0; i < encoders_.size(); ++i) { - ARROW_ASSIGN_OR_RAISE( - out.values[i], - encoders_[i]->Decode(key_buf_ptrs.data(), static_cast(num_groups_), - ctx_->memory_pool())); - } - - return out; - } - - ExecContext* ctx_; - std::unordered_map map_; - std::vector offsets_ = {0}; - std::vector key_bytes_; - uint32_t num_groups_ = 0; - std::vector> encoders_; -}; - -struct GrouperFastImpl : Grouper { - static constexpr int kBitmapPaddingForSIMD = 64; // bits - static constexpr int kPaddingForSIMD = 32; // bytes - - static bool CanUse(const std::vector& keys) { -#if ARROW_LITTLE_ENDIAN - for (size_t i = 0; i < keys.size(); ++i) { - const auto& key = keys[i].type; - if (is_large_binary_like(key->id())) { - return false; - } - } - return true; -#else - return false; -#endif - } - - static Result> Make( - const std::vector& keys, ExecContext* ctx) { - auto impl = ::arrow::internal::make_unique(); - impl->ctx_ = ctx; - - RETURN_NOT_OK(impl->temp_stack_.Init(ctx->memory_pool(), 64 * minibatch_size_max_)); - impl->encode_ctx_.hardware_flags = - arrow::internal::CpuInfo::GetInstance()->hardware_flags(); - impl->encode_ctx_.stack = &impl->temp_stack_; - - auto num_columns = keys.size(); - impl->col_metadata_.resize(num_columns); - impl->key_types_.resize(num_columns); - impl->dictionaries_.resize(num_columns); - for (size_t icol = 0; icol < num_columns; ++icol) { - const auto& key = keys[icol].type; - if (key->id() == Type::DICTIONARY) { - auto bit_width = checked_cast(*key).bit_width(); - ARROW_DCHECK(bit_width % 8 == 0); - impl->col_metadata_[icol] = - arrow::compute::KeyColumnMetadata(true, bit_width / 8); - } else if (key->id() == Type::BOOL) { - impl->col_metadata_[icol] = arrow::compute::KeyColumnMetadata(true, 0); - } else if (is_fixed_width(key->id())) { - impl->col_metadata_[icol] = arrow::compute::KeyColumnMetadata( - true, checked_cast(*key).bit_width() / 8); - } else if (is_binary_like(key->id())) { - impl->col_metadata_[icol] = - arrow::compute::KeyColumnMetadata(false, sizeof(uint32_t)); - } else if (key->id() == Type::NA) { - impl->col_metadata_[icol] = - arrow::compute::KeyColumnMetadata(true, 0, /*is_null_type_in=*/true); - } else { - return Status::NotImplemented("Keys of type ", *key); - } - impl->key_types_[icol] = key; - } - - impl->encoder_.Init(impl->col_metadata_, &impl->encode_ctx_, - /* row_alignment = */ sizeof(uint64_t), - /* string_alignment = */ sizeof(uint64_t)); - RETURN_NOT_OK(impl->rows_.Init(ctx->memory_pool(), impl->encoder_.row_metadata())); - RETURN_NOT_OK( - impl->rows_minibatch_.Init(ctx->memory_pool(), impl->encoder_.row_metadata())); - impl->minibatch_size_ = impl->minibatch_size_min_; - GrouperFastImpl* impl_ptr = impl.get(); - auto equal_func = [impl_ptr]( - int num_keys_to_compare, const uint16_t* selection_may_be_null, - const uint32_t* group_ids, uint32_t* out_num_keys_mismatch, - uint16_t* out_selection_mismatch) { - arrow::compute::KeyCompare::CompareColumnsToRows( - num_keys_to_compare, selection_may_be_null, group_ids, &impl_ptr->encode_ctx_, - out_num_keys_mismatch, out_selection_mismatch, - impl_ptr->encoder_.GetBatchColumns(), impl_ptr->rows_); - }; - auto append_func = [impl_ptr](int num_keys, const uint16_t* selection) { - RETURN_NOT_OK(impl_ptr->encoder_.EncodeSelected(&impl_ptr->rows_minibatch_, - num_keys, selection)); - return impl_ptr->rows_.AppendSelectionFrom(impl_ptr->rows_minibatch_, num_keys, - nullptr); - }; - RETURN_NOT_OK(impl->map_.init(impl->encode_ctx_.hardware_flags, ctx->memory_pool(), - impl->encode_ctx_.stack, impl->log_minibatch_max_, - equal_func, append_func)); - impl->cols_.resize(num_columns); - impl->minibatch_hashes_.resize(impl->minibatch_size_max_ + - kPaddingForSIMD / sizeof(uint32_t)); - - return std::move(impl); - } - - ~GrouperFastImpl() { map_.cleanup(); } - - Result Consume(const ExecBatch& batch) override { - // ARROW-14027: broadcast scalar arguments for now - for (int i = 0; i < batch.num_values(); i++) { - if (batch.values[i].is_scalar()) { - ExecBatch expanded = batch; - for (int j = i; j < expanded.num_values(); j++) { - if (expanded.values[j].is_scalar()) { - ARROW_ASSIGN_OR_RAISE( - expanded.values[j], - MakeArrayFromScalar(*expanded.values[j].scalar(), expanded.length, - ctx_->memory_pool())); - } - } - return ConsumeImpl(expanded); - } - } - return ConsumeImpl(batch); - } - - Result ConsumeImpl(const ExecBatch& batch) { - int64_t num_rows = batch.length; - int num_columns = batch.num_values(); - // Process dictionaries - for (int icol = 0; icol < num_columns; ++icol) { - if (key_types_[icol]->id() == Type::DICTIONARY) { - auto data = batch[icol].array(); - auto dict = MakeArray(data->dictionary); - if (dictionaries_[icol]) { - if (!dictionaries_[icol]->Equals(dict)) { - // TODO(bkietz) unify if necessary. For now, just error if any batch's - // dictionary differs from the first we saw for this key - return Status::NotImplemented("Unifying differing dictionaries"); - } - } else { - dictionaries_[icol] = std::move(dict); - } - } - } - - std::shared_ptr group_ids; - ARROW_ASSIGN_OR_RAISE( - group_ids, AllocateBuffer(sizeof(uint32_t) * num_rows, ctx_->memory_pool())); - - for (int icol = 0; icol < num_columns; ++icol) { - const uint8_t* non_nulls = NULLPTR; - const uint8_t* fixedlen = NULLPTR; - const uint8_t* varlen = NULLPTR; - - // Skip if the key's type is NULL - if (key_types_[icol]->id() != Type::NA) { - if (batch[icol].array()->buffers[0] != NULLPTR) { - non_nulls = batch[icol].array()->buffers[0]->data(); - } - fixedlen = batch[icol].array()->buffers[1]->data(); - if (!col_metadata_[icol].is_fixed_length) { - varlen = batch[icol].array()->buffers[2]->data(); - } - } - - int64_t offset = batch[icol].array()->offset; - - auto col_base = arrow::compute::KeyColumnArray( - col_metadata_[icol], offset + num_rows, non_nulls, fixedlen, varlen); - - cols_[icol] = col_base.Slice(offset, num_rows); - } - - // Split into smaller mini-batches - // - for (uint32_t start_row = 0; start_row < num_rows;) { - uint32_t batch_size_next = std::min(static_cast(minibatch_size_), - static_cast(num_rows) - start_row); - - // Encode - rows_minibatch_.Clean(); - encoder_.PrepareEncodeSelected(start_row, batch_size_next, cols_); - - // Compute hash - Hashing32::HashMultiColumn(encoder_.GetBatchColumns(), &encode_ctx_, - minibatch_hashes_.data()); - - // Map - auto match_bitvector = - util::TempVectorHolder(&temp_stack_, (batch_size_next + 7) / 8); - { - auto local_slots = util::TempVectorHolder(&temp_stack_, batch_size_next); - map_.early_filter(batch_size_next, minibatch_hashes_.data(), - match_bitvector.mutable_data(), local_slots.mutable_data()); - map_.find(batch_size_next, minibatch_hashes_.data(), - match_bitvector.mutable_data(), local_slots.mutable_data(), - reinterpret_cast(group_ids->mutable_data()) + start_row); - } - auto ids = util::TempVectorHolder(&temp_stack_, batch_size_next); - int num_ids; - util::bit_util::bits_to_indexes(0, encode_ctx_.hardware_flags, batch_size_next, - match_bitvector.mutable_data(), &num_ids, - ids.mutable_data()); - - RETURN_NOT_OK(map_.map_new_keys( - num_ids, ids.mutable_data(), minibatch_hashes_.data(), - reinterpret_cast(group_ids->mutable_data()) + start_row)); - - start_row += batch_size_next; - - if (minibatch_size_ * 2 <= minibatch_size_max_) { - minibatch_size_ *= 2; - } - } - - return Datum(UInt32Array(batch.length, std::move(group_ids))); - } - - uint32_t num_groups() const override { return static_cast(rows_.length()); } - - // Make sure padded buffers end up with the right logical size - - Result> AllocatePaddedBitmap(int64_t length) { - ARROW_ASSIGN_OR_RAISE( - std::shared_ptr buf, - AllocateBitmap(length + kBitmapPaddingForSIMD, ctx_->memory_pool())); - return SliceMutableBuffer(buf, 0, bit_util::BytesForBits(length)); - } - - Result> AllocatePaddedBuffer(int64_t size) { - ARROW_ASSIGN_OR_RAISE( - std::shared_ptr buf, - AllocateBuffer(size + kBitmapPaddingForSIMD, ctx_->memory_pool())); - return SliceMutableBuffer(buf, 0, size); - } - - Result GetUniques() override { - auto num_columns = static_cast(col_metadata_.size()); - int64_t num_groups = rows_.length(); - - std::vector> non_null_bufs(num_columns); - std::vector> fixedlen_bufs(num_columns); - std::vector> varlen_bufs(num_columns); - - for (size_t i = 0; i < num_columns; ++i) { - if (col_metadata_[i].is_null_type) { - uint8_t* non_nulls = NULLPTR; - uint8_t* fixedlen = NULLPTR; - cols_[i] = arrow::compute::KeyColumnArray(col_metadata_[i], num_groups, non_nulls, - fixedlen, NULLPTR); - continue; - } - ARROW_ASSIGN_OR_RAISE(non_null_bufs[i], AllocatePaddedBitmap(num_groups)); - if (col_metadata_[i].is_fixed_length && !col_metadata_[i].is_null_type) { - if (col_metadata_[i].fixed_length == 0) { - ARROW_ASSIGN_OR_RAISE(fixedlen_bufs[i], AllocatePaddedBitmap(num_groups)); - } else { - ARROW_ASSIGN_OR_RAISE( - fixedlen_bufs[i], - AllocatePaddedBuffer(num_groups * col_metadata_[i].fixed_length)); - } - } else { - ARROW_ASSIGN_OR_RAISE(fixedlen_bufs[i], - AllocatePaddedBuffer((num_groups + 1) * sizeof(uint32_t))); - } - cols_[i] = arrow::compute::KeyColumnArray( - col_metadata_[i], num_groups, non_null_bufs[i]->mutable_data(), - fixedlen_bufs[i]->mutable_data(), nullptr); - } - - for (int64_t start_row = 0; start_row < num_groups;) { - int64_t batch_size_next = - std::min(num_groups - start_row, static_cast(minibatch_size_max_)); - encoder_.DecodeFixedLengthBuffers(start_row, start_row, batch_size_next, rows_, - &cols_); - start_row += batch_size_next; - } - - if (!rows_.metadata().is_fixed_length) { - for (size_t i = 0; i < num_columns; ++i) { - if (!col_metadata_[i].is_fixed_length) { - auto varlen_size = - reinterpret_cast(fixedlen_bufs[i]->data())[num_groups]; - ARROW_ASSIGN_OR_RAISE(varlen_bufs[i], AllocatePaddedBuffer(varlen_size)); - cols_[i] = arrow::compute::KeyColumnArray( - col_metadata_[i], num_groups, non_null_bufs[i]->mutable_data(), - fixedlen_bufs[i]->mutable_data(), varlen_bufs[i]->mutable_data()); - } - } - - for (int64_t start_row = 0; start_row < num_groups;) { - int64_t batch_size_next = - std::min(num_groups - start_row, static_cast(minibatch_size_max_)); - encoder_.DecodeVaryingLengthBuffers(start_row, start_row, batch_size_next, rows_, - &cols_); - start_row += batch_size_next; - } - } - - ExecBatch out({}, num_groups); - out.values.resize(num_columns); - for (size_t i = 0; i < num_columns; ++i) { - if (col_metadata_[i].is_null_type) { - out.values[i] = ArrayData::Make(null(), num_groups, {nullptr}, num_groups); - continue; - } - auto valid_count = arrow::internal::CountSetBits( - non_null_bufs[i]->data(), /*offset=*/0, static_cast(num_groups)); - int null_count = static_cast(num_groups) - static_cast(valid_count); - - if (col_metadata_[i].is_fixed_length) { - out.values[i] = ArrayData::Make( - key_types_[i], num_groups, - {std::move(non_null_bufs[i]), std::move(fixedlen_bufs[i])}, null_count); - } else { - out.values[i] = - ArrayData::Make(key_types_[i], num_groups, - {std::move(non_null_bufs[i]), std::move(fixedlen_bufs[i]), - std::move(varlen_bufs[i])}, - null_count); - } - } - - // Process dictionaries - for (size_t icol = 0; icol < num_columns; ++icol) { - if (key_types_[icol]->id() == Type::DICTIONARY) { - if (dictionaries_[icol]) { - out.values[icol].array()->dictionary = dictionaries_[icol]->data(); - } else { - ARROW_ASSIGN_OR_RAISE(auto dict, MakeArrayOfNull(key_types_[icol], 0)); - out.values[icol].array()->dictionary = dict->data(); - } - } - } - - return out; - } - - static constexpr int log_minibatch_max_ = 10; - static constexpr int minibatch_size_max_ = 1 << log_minibatch_max_; - static constexpr int minibatch_size_min_ = 128; - int minibatch_size_; - - ExecContext* ctx_; - arrow::util::TempVectorStack temp_stack_; - arrow::compute::KeyEncoder::KeyEncoderContext encode_ctx_; - - std::vector> key_types_; - std::vector col_metadata_; - std::vector cols_; - std::vector minibatch_hashes_; - - std::vector> dictionaries_; - - arrow::compute::KeyEncoder::KeyRowArray rows_; - arrow::compute::KeyEncoder::KeyRowArray rows_minibatch_; - arrow::compute::KeyEncoder encoder_; - arrow::compute::SwissTable map_; -}; - /// C++ abstract base class for the HashAggregateKernel interface. /// Implementations should be default constructible and perform initialization in /// Init(). @@ -3244,14 +2764,6 @@ Result ResolveKernels( return fields; } -Result> Grouper::Make(const std::vector& descrs, - ExecContext* ctx) { - if (GrouperFastImpl::CanUse(descrs)) { - return GrouperFastImpl::Make(descrs, ctx); - } - return GrouperImpl::Make(descrs, ctx); -} - Result GroupBy(const std::vector& arguments, const std::vector& keys, const std::vector& aggregates, bool use_threads, ExecContext* ctx) { @@ -3385,58 +2897,6 @@ Result GroupBy(const std::vector& arguments, const std::vector> Grouper::ApplyGroupings(const ListArray& groupings, - const Array& array, - ExecContext* ctx) { - ARROW_ASSIGN_OR_RAISE(Datum sorted, - compute::Take(array, groupings.data()->child_data[0], - TakeOptions::NoBoundsCheck(), ctx)); - - return std::make_shared(list(array.type()), groupings.length(), - groupings.value_offsets(), sorted.make_array()); -} - -Result> Grouper::MakeGroupings(const UInt32Array& ids, - uint32_t num_groups, - ExecContext* ctx) { - if (ids.null_count() != 0) { - return Status::Invalid("MakeGroupings with null ids"); - } - - ARROW_ASSIGN_OR_RAISE(auto offsets, AllocateBuffer(sizeof(int32_t) * (num_groups + 1), - ctx->memory_pool())); - auto raw_offsets = reinterpret_cast(offsets->mutable_data()); - - std::memset(raw_offsets, 0, offsets->size()); - for (int i = 0; i < ids.length(); ++i) { - DCHECK_LT(ids.Value(i), num_groups); - raw_offsets[ids.Value(i)] += 1; - } - int32_t length = 0; - for (uint32_t id = 0; id < num_groups; ++id) { - auto offset = raw_offsets[id]; - raw_offsets[id] = length; - length += offset; - } - raw_offsets[num_groups] = length; - DCHECK_EQ(ids.length(), length); - - ARROW_ASSIGN_OR_RAISE(auto offsets_copy, - offsets->CopySlice(0, offsets->size(), ctx->memory_pool())); - raw_offsets = reinterpret_cast(offsets_copy->mutable_data()); - - ARROW_ASSIGN_OR_RAISE(auto sort_indices, AllocateBuffer(sizeof(int32_t) * ids.length(), - ctx->memory_pool())); - auto raw_sort_indices = reinterpret_cast(sort_indices->mutable_data()); - for (int i = 0; i < ids.length(); ++i) { - raw_sort_indices[raw_offsets[ids.Value(i)]++] = i; - } - - return std::make_shared( - list(int32()), num_groups, std::move(offsets), - std::make_shared(ids.length(), std::move(sort_indices))); -} - namespace { const FunctionDoc hash_count_doc{ "Count the number of null / non-null values in each group", diff --git a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc index 35b4592e198..b7c3c1eb86a 100644 --- a/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc +++ b/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc @@ -41,6 +41,7 @@ #include "arrow/compute/kernels/codegen_internal.h" #include "arrow/compute/kernels/test_util.h" #include "arrow/compute/registry.h" +#include "arrow/compute/row/grouper.h" #include "arrow/table.h" #include "arrow/testing/generator.h" #include "arrow/testing/gtest_util.h" @@ -73,14 +74,13 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys const std::vector& aggregates) { ARROW_ASSIGN_OR_RAISE(auto key_batch, ExecBatch::Make(std::move(keys))); - ARROW_ASSIGN_OR_RAISE(auto grouper, - internal::Grouper::Make(key_batch.GetDescriptors())); + ARROW_ASSIGN_OR_RAISE(auto grouper, Grouper::Make(key_batch.GetDescriptors())); ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); ARROW_ASSIGN_OR_RAISE( - auto groupings, internal::Grouper::MakeGroupings(*id_batch.array_as(), - grouper->num_groups())); + auto groupings, + Grouper::MakeGroupings(*id_batch.array_as(), grouper->num_groups())); ArrayVector out_columns; std::vector out_names; @@ -93,7 +93,7 @@ Result NaiveGroupBy(std::vector arguments, std::vector keys ARROW_ASSIGN_OR_RAISE( auto grouped_argument, - internal::Grouper::ApplyGroupings(*groupings, *arguments[i].make_array())); + Grouper::ApplyGroupings(*groupings, *arguments[i].make_array())); ScalarVector aggregated_scalars; @@ -261,21 +261,21 @@ Result GroupByTest( } // namespace TEST(Grouper, SupportedKeys) { - ASSERT_OK(internal::Grouper::Make({boolean()})); + ASSERT_OK(Grouper::Make({boolean()})); - ASSERT_OK(internal::Grouper::Make({int8(), uint16(), int32(), uint64()})); + ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()})); - ASSERT_OK(internal::Grouper::Make({dictionary(int64(), utf8())})); + ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())})); - ASSERT_OK(internal::Grouper::Make({float16(), float32(), float64()})); + ASSERT_OK(Grouper::Make({float16(), float32(), float64()})); - ASSERT_OK(internal::Grouper::Make({utf8(), binary(), large_utf8(), large_binary()})); + ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()})); - ASSERT_OK(internal::Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)})); + ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)})); - ASSERT_OK(internal::Grouper::Make({decimal128(32, 10), decimal256(76, 20)})); + ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)})); - ASSERT_OK(internal::Grouper::Make({date32(), date64()})); + ASSERT_OK(Grouper::Make({date32(), date64()})); for (auto unit : { TimeUnit::SECOND, @@ -283,29 +283,28 @@ TEST(Grouper, SupportedKeys) { TimeUnit::MICRO, TimeUnit::NANO, }) { - ASSERT_OK(internal::Grouper::Make({timestamp(unit), duration(unit)})); + ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)})); } - ASSERT_OK(internal::Grouper::Make( - {day_time_interval(), month_interval(), month_day_nano_interval()})); + ASSERT_OK( + Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()})); - ASSERT_OK(internal::Grouper::Make({null()})); + ASSERT_OK(Grouper::Make({null()})); - ASSERT_RAISES(NotImplemented, internal::Grouper::Make({struct_({field("", int64())})})); + ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({field("", int64())})})); - ASSERT_RAISES(NotImplemented, internal::Grouper::Make({struct_({})})); + ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({})})); - ASSERT_RAISES(NotImplemented, internal::Grouper::Make({list(int32())})); + ASSERT_RAISES(NotImplemented, Grouper::Make({list(int32())})); - ASSERT_RAISES(NotImplemented, internal::Grouper::Make({fixed_size_list(int32(), 5)})); + ASSERT_RAISES(NotImplemented, Grouper::Make({fixed_size_list(int32(), 5)})); - ASSERT_RAISES(NotImplemented, - internal::Grouper::Make({dense_union({field("", int32())})})); + ASSERT_RAISES(NotImplemented, Grouper::Make({dense_union({field("", int32())})})); } struct TestGrouper { explicit TestGrouper(std::vector descrs) : descrs_(std::move(descrs)) { - grouper_ = internal::Grouper::Make(descrs_).ValueOrDie(); + grouper_ = Grouper::Make(descrs_).ValueOrDie(); FieldVector fields; for (const auto& descr : descrs_) { @@ -423,7 +422,7 @@ struct TestGrouper { std::vector descrs_; std::shared_ptr key_schema_; - std::unique_ptr grouper_; + std::unique_ptr grouper_; ExecBatch uniques_ = ExecBatch({}, -1); }; @@ -666,12 +665,11 @@ TEST(Grouper, MakeGroupings) { auto expected = ArrayFromJSON(list(int32()), expected_json); auto num_groups = static_cast(expected->length()); - ASSERT_OK_AND_ASSIGN(auto actual, internal::Grouper::MakeGroupings(*ids, num_groups)); + ASSERT_OK_AND_ASSIGN(auto actual, Grouper::MakeGroupings(*ids, num_groups)); AssertArraysEqual(*expected, *actual, /*verbose=*/true); // validate ApplyGroupings - ASSERT_OK_AND_ASSIGN(auto grouped_ids, - internal::Grouper::ApplyGroupings(*actual, *ids)); + ASSERT_OK_AND_ASSIGN(auto grouped_ids, Grouper::ApplyGroupings(*actual, *ids)); for (uint32_t group = 0; group < num_groups; ++group) { auto ids_slice = checked_pointer_cast(grouped_ids->value_slice(group)); @@ -693,7 +691,7 @@ TEST(Grouper, MakeGroupings) { auto ids = checked_pointer_cast(ArrayFromJSON(uint32(), "[0, null, 1]")); EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("MakeGroupings with null ids"), - internal::Grouper::MakeGroupings(*ids, 5)); + Grouper::MakeGroupings(*ids, 5)); } TEST(Grouper, ScalarValues) { diff --git a/cpp/src/arrow/compute/light_array.h b/cpp/src/arrow/compute/light_array.h index dd13aa0647f..f0e5c706871 100644 --- a/cpp/src/arrow/compute/light_array.h +++ b/cpp/src/arrow/compute/light_array.h @@ -21,7 +21,9 @@ #include "arrow/array.h" #include "arrow/compute/exec.h" +#include "arrow/compute/exec/util.h" #include "arrow/type.h" +#include "arrow/util/cpu_info.h" #include "arrow/util/logging.h" /// This file contains lightweight containers for Arrow buffers. These containers @@ -31,6 +33,18 @@ namespace arrow { namespace compute { +/// \brief Context needed by various execution engine operations +/// +/// In the execution engine this context is provided by either the node or the +/// plan and the context exists for the lifetime of the plan. Defining this here +/// allows us to take advantage of these resources without coupling the logic with +/// the execution engine. +struct LightContext { + bool has_avx2() const { return (hardware_flags & arrow::internal::CpuInfo::AVX2) > 0; } + int64_t hardware_flags; + util::TempVectorStack* stack; +}; + /// \brief Description of the layout of a "key" column /// /// A "key" column is a non-nested, non-union column. diff --git a/cpp/src/arrow/compute/row/CMakeLists.txt b/cpp/src/arrow/compute/row/CMakeLists.txt new file mode 100644 index 00000000000..6ae982dbaf3 --- /dev/null +++ b/cpp/src/arrow/compute/row/CMakeLists.txt @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# Contains utilities for working with Arrow data been stored +# in a row-major order. + +arrow_install_all_headers("arrow/compute/row") diff --git a/cpp/src/arrow/compute/exec/key_compare.cc b/cpp/src/arrow/compute/row/compare_internal.cc similarity index 90% rename from cpp/src/arrow/compute/exec/key_compare.cc rename to cpp/src/arrow/compute/row/compare_internal.cc index d873aec692e..e863c9cd05f 100644 --- a/cpp/src/arrow/compute/exec/key_compare.cc +++ b/cpp/src/arrow/compute/row/compare_internal.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/key_compare.h" +#include "arrow/compute/row/compare_internal.h" #include @@ -33,9 +33,8 @@ template void KeyCompare::NullUpdateColumnToRow(uint32_t id_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, + LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector) { if (!rows.has_any_nulls(ctx) && !col.data(0)) { return; @@ -90,9 +89,8 @@ template void KeyCompare::CompareBinaryColumnToRowHelper( uint32_t offset_within_row, uint32_t first_row_to_compare, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, const KeyEncoder::KeyRowArray& rows, - uint8_t* match_bytevector, COMPARE_FN compare_fn) { + const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector, COMPARE_FN compare_fn) { bool is_fixed_length = rows.metadata().is_fixed_length; if (is_fixed_length) { uint32_t fixed_length = rows.metadata().fixed_length; @@ -118,11 +116,13 @@ void KeyCompare::CompareBinaryColumnToRowHelper( } template -void KeyCompare::CompareBinaryColumnToRow( - uint32_t offset_within_row, uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector) { +void KeyCompare::CompareBinaryColumnToRow(uint32_t offset_within_row, + uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, + LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, + uint8_t* match_bytevector) { uint32_t num_processed = 0; #if defined(ARROW_HAVE_AVX2) if (ctx->has_avx2()) { @@ -228,11 +228,13 @@ void KeyCompare::CompareBinaryColumnToRow( // Overwrites the match_bytevector instead of updating it template -void KeyCompare::CompareVarBinaryColumnToRow( - uint32_t id_varbinary_col, uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector) { +void KeyCompare::CompareVarBinaryColumnToRow(uint32_t id_varbinary_col, + uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, + LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, + uint8_t* match_bytevector) { #if defined(ARROW_HAVE_AVX2) if (ctx->has_avx2()) { CompareVarBinaryColumnToRow_avx2( @@ -290,7 +292,7 @@ void KeyCompare::CompareVarBinaryColumnToRow( } } -void KeyCompare::AndByteVectors(KeyEncoder::KeyEncoderContext* ctx, uint32_t num_elements, +void KeyCompare::AndByteVectors(LightContext* ctx, uint32_t num_elements, uint8_t* bytevector_A, const uint8_t* bytevector_B) { uint32_t num_processed = 0; #if defined(ARROW_HAVE_AVX2) @@ -306,11 +308,13 @@ void KeyCompare::AndByteVectors(KeyEncoder::KeyEncoderContext* ctx, uint32_t num } } -void KeyCompare::CompareColumnsToRows( - uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - uint32_t* out_num_rows, uint16_t* out_sel_left_maybe_same, - const std::vector& cols, const KeyEncoder::KeyRowArray& rows) { +void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, + LightContext* ctx, uint32_t* out_num_rows, + uint16_t* out_sel_left_maybe_same, + const std::vector& cols, + const RowTableImpl& rows) { if (num_rows_to_compare == 0) { *out_num_rows = 0; return; diff --git a/cpp/src/arrow/compute/exec/key_compare.h b/cpp/src/arrow/compute/row/compare_internal.h similarity index 52% rename from cpp/src/arrow/compute/exec/key_compare.h rename to cpp/src/arrow/compute/row/compare_internal.h index 773b32d46c6..e3b9057115e 100644 --- a/cpp/src/arrow/compute/exec/key_compare.h +++ b/cpp/src/arrow/compute/row/compare_internal.h @@ -19,8 +19,10 @@ #include -#include "arrow/compute/exec/key_encode.h" #include "arrow/compute/exec/util.h" +#include "arrow/compute/light_array.h" +#include "arrow/compute/row/encode_internal.h" +#include "arrow/compute/row/row_internal.h" #include "arrow/memory_pool.h" #include "arrow/result.h" #include "arrow/status.h" @@ -33,45 +35,48 @@ class KeyCompare { // Returns a single 16-bit selection vector of rows that failed comparison. // If there is input selection on the left, the resulting selection is a filtered image // of input selection. - static void CompareColumnsToRows( - uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - uint32_t* out_num_rows, uint16_t* out_sel_left_maybe_same, - const std::vector& cols, const KeyEncoder::KeyRowArray& rows); + static void CompareColumnsToRows(uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, LightContext* ctx, + uint32_t* out_num_rows, + uint16_t* out_sel_left_maybe_same, + const std::vector& cols, + const RowTableImpl& rows); private: template static void NullUpdateColumnToRow(uint32_t id_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, + const uint32_t* left_to_right_map, LightContext* ctx, + const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector); template static void CompareBinaryColumnToRowHelper( uint32_t offset_within_row, uint32_t first_row_to_compare, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, const KeyEncoder::KeyRowArray& rows, - uint8_t* match_bytevector, COMPARE_FN compare_fn); + const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector, COMPARE_FN compare_fn); template - static void CompareBinaryColumnToRow( - uint32_t offset_within_row, uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector); + static void CompareBinaryColumnToRow(uint32_t offset_within_row, + uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, + LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, + uint8_t* match_bytevector); template - static void CompareVarBinaryColumnToRow( - uint32_t id_varlen_col, uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector); - - static void AndByteVectors(KeyEncoder::KeyEncoderContext* ctx, uint32_t num_elements, + static void CompareVarBinaryColumnToRow(uint32_t id_varlen_col, + uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, + LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, + uint8_t* match_bytevector); + + static void AndByteVectors(LightContext* ctx, uint32_t num_elements, uint8_t* bytevector_A, const uint8_t* bytevector_B); #if defined(ARROW_HAVE_AVX2) @@ -79,53 +84,52 @@ class KeyCompare { template static uint32_t NullUpdateColumnToRowImp_avx2( uint32_t id_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, const KeyEncoder::KeyRowArray& rows, - uint8_t* match_bytevector); + const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector); template static uint32_t CompareBinaryColumnToRowHelper_avx2( uint32_t offset_within_row, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector, - COMPARE8_FN compare8_fn); + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector, COMPARE8_FN compare8_fn); template static uint32_t CompareBinaryColumnToRowImp_avx2( uint32_t offset_within_row, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector); + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector); template static void CompareVarBinaryColumnToRowImp_avx2( uint32_t id_varlen_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector); + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector); static uint32_t AndByteVectors_avx2(uint32_t num_elements, uint8_t* bytevector_A, const uint8_t* bytevector_B); - static uint32_t NullUpdateColumnToRow_avx2( - bool use_selection, uint32_t id_col, uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector); + static uint32_t NullUpdateColumnToRow_avx2(bool use_selection, uint32_t id_col, + uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, + LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, + uint8_t* match_bytevector); static uint32_t CompareBinaryColumnToRow_avx2( bool use_selection, uint32_t offset_within_row, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector); + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector); static void CompareVarBinaryColumnToRow_avx2( bool use_selection, bool is_first_varbinary_col, uint32_t id_varlen_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, const KeyEncoder::KeyRowArray& rows, - uint8_t* match_bytevector); + const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector); #endif }; diff --git a/cpp/src/arrow/compute/exec/key_compare_avx2.cc b/cpp/src/arrow/compute/row/compare_internal_avx2.cc similarity index 96% rename from cpp/src/arrow/compute/exec/key_compare_avx2.cc rename to cpp/src/arrow/compute/row/compare_internal_avx2.cc index e45486b2ebb..818f4c4fe7f 100644 --- a/cpp/src/arrow/compute/exec/key_compare_avx2.cc +++ b/cpp/src/arrow/compute/row/compare_internal_avx2.cc @@ -17,7 +17,7 @@ #include -#include "arrow/compute/exec/key_compare.h" +#include "arrow/compute/row/compare_internal.h" #include "arrow/util/bit_util.h" namespace arrow { @@ -39,9 +39,8 @@ inline __m256i set_first_n_bytes_avx2(int n) { template uint32_t KeyCompare::NullUpdateColumnToRowImp_avx2( uint32_t id_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, const KeyEncoder::KeyRowArray& rows, - uint8_t* match_bytevector) { + const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector) { if (!rows.has_any_nulls(ctx) && !col.data(0)) { return num_rows_to_compare; } @@ -180,9 +179,8 @@ template uint32_t KeyCompare::CompareBinaryColumnToRowHelper_avx2( uint32_t offset_within_row, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector, - COMPARE8_FN compare8_fn) { + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector, COMPARE8_FN compare8_fn) { bool is_fixed_length = rows.metadata().is_fixed_length; if (is_fixed_length) { uint32_t fixed_length = rows.metadata().fixed_length; @@ -419,8 +417,8 @@ template uint32_t KeyCompare::CompareBinaryColumnToRowImp_avx2( uint32_t offset_within_row, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector) { + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector) { uint32_t col_width = col.metadata().fixed_length; if (col_width == 0) { int bit_offset = col.bit_offset(1); @@ -503,8 +501,8 @@ template void KeyCompare::CompareVarBinaryColumnToRowImp_avx2( uint32_t id_varbinary_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector) { + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector) { const uint32_t* offsets_left = col.offsets(); const uint32_t* offsets_right = rows.offsets(); const uint8_t* rows_left = col.data(2); @@ -569,8 +567,8 @@ uint32_t KeyCompare::AndByteVectors_avx2(uint32_t num_elements, uint8_t* bytevec uint32_t KeyCompare::NullUpdateColumnToRow_avx2( bool use_selection, uint32_t id_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector) { + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector) { if (use_selection) { return NullUpdateColumnToRowImp_avx2(id_col, num_rows_to_compare, sel_left_maybe_null, left_to_right_map, @@ -585,8 +583,8 @@ uint32_t KeyCompare::NullUpdateColumnToRow_avx2( uint32_t KeyCompare::CompareBinaryColumnToRow_avx2( bool use_selection, uint32_t offset_within_row, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, - KeyEncoder::KeyEncoderContext* ctx, const KeyColumnArray& col, - const KeyEncoder::KeyRowArray& rows, uint8_t* match_bytevector) { + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector) { if (use_selection) { return CompareBinaryColumnToRowImp_avx2(offset_within_row, num_rows_to_compare, sel_left_maybe_null, left_to_right_map, @@ -601,9 +599,8 @@ uint32_t KeyCompare::CompareBinaryColumnToRow_avx2( void KeyCompare::CompareVarBinaryColumnToRow_avx2( bool use_selection, bool is_first_varbinary_col, uint32_t id_varlen_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, KeyEncoder::KeyEncoderContext* ctx, - const KeyColumnArray& col, const KeyEncoder::KeyRowArray& rows, - uint8_t* match_bytevector) { + const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector) { if (use_selection) { if (is_first_varbinary_col) { CompareVarBinaryColumnToRowImp_avx2( diff --git a/cpp/src/arrow/compute/exec/key_encode.cc b/cpp/src/arrow/compute/row/encode_internal.cc similarity index 56% rename from cpp/src/arrow/compute/exec/key_encode.cc rename to cpp/src/arrow/compute/row/encode_internal.cc index 3d92c77b09c..cbfd169b448 100644 --- a/cpp/src/arrow/compute/exec/key_encode.cc +++ b/cpp/src/arrow/compute/row/encode_internal.cc @@ -15,318 +15,223 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/compute/exec/key_encode.h" - -#include - -#include - -#include "arrow/compute/exec/util.h" -#include "arrow/util/bit_util.h" -#include "arrow/util/ubsan.h" +#include "arrow/compute/row/encode_internal.h" namespace arrow { namespace compute { -KeyEncoder::KeyRowArray::KeyRowArray() - : pool_(nullptr), rows_capacity_(0), bytes_capacity_(0) {} - -Status KeyEncoder::KeyRowArray::Init(MemoryPool* pool, const KeyRowMetadata& metadata) { - pool_ = pool; - metadata_ = metadata; - - DCHECK(!null_masks_ && !offsets_ && !rows_); - - constexpr int64_t rows_capacity = 8; - constexpr int64_t bytes_capacity = 1024; - - // Null masks - ARROW_ASSIGN_OR_RAISE(auto null_masks, - AllocateResizableBuffer(size_null_masks(rows_capacity), pool_)); - null_masks_ = std::move(null_masks); - memset(null_masks_->mutable_data(), 0, size_null_masks(rows_capacity)); - - // Offsets and rows - if (!metadata.is_fixed_length) { - ARROW_ASSIGN_OR_RAISE(auto offsets, - AllocateResizableBuffer(size_offsets(rows_capacity), pool_)); - offsets_ = std::move(offsets); - memset(offsets_->mutable_data(), 0, size_offsets(rows_capacity)); - reinterpret_cast(offsets_->mutable_data())[0] = 0; - - ARROW_ASSIGN_OR_RAISE( - auto rows, - AllocateResizableBuffer(size_rows_varying_length(bytes_capacity), pool_)); - rows_ = std::move(rows); - memset(rows_->mutable_data(), 0, size_rows_varying_length(bytes_capacity)); - bytes_capacity_ = size_rows_varying_length(bytes_capacity) - padding_for_vectors; - } else { - ARROW_ASSIGN_OR_RAISE( - auto rows, AllocateResizableBuffer(size_rows_fixed_length(rows_capacity), pool_)); - rows_ = std::move(rows); - memset(rows_->mutable_data(), 0, size_rows_fixed_length(rows_capacity)); - bytes_capacity_ = size_rows_fixed_length(rows_capacity) - padding_for_vectors; - } - - update_buffer_pointers(); - - rows_capacity_ = rows_capacity; - - num_rows_ = 0; - num_rows_for_has_any_nulls_ = 0; - has_any_nulls_ = false; - - return Status::OK(); -} - -void KeyEncoder::KeyRowArray::Clean() { - num_rows_ = 0; - num_rows_for_has_any_nulls_ = 0; - has_any_nulls_ = false; - - if (!metadata_.is_fixed_length) { - reinterpret_cast(offsets_->mutable_data())[0] = 0; - } -} - -int64_t KeyEncoder::KeyRowArray::size_null_masks(int64_t num_rows) { - return num_rows * metadata_.null_masks_bytes_per_row + padding_for_vectors; -} - -int64_t KeyEncoder::KeyRowArray::size_offsets(int64_t num_rows) { - return (num_rows + 1) * sizeof(uint32_t) + padding_for_vectors; +void RowTableEncoder::Init(const std::vector& cols, LightContext* ctx, + int row_alignment, int string_alignment) { + ctx_ = ctx; + row_metadata_.FromColumnMetadataVector(cols, row_alignment, string_alignment); + uint32_t num_cols = row_metadata_.num_cols(); + uint32_t num_varbinary_cols = row_metadata_.num_varbinary_cols(); + batch_all_cols_.resize(num_cols); + batch_varbinary_cols_.resize(num_varbinary_cols); + batch_varbinary_cols_base_offsets_.resize(num_varbinary_cols); } -int64_t KeyEncoder::KeyRowArray::size_rows_fixed_length(int64_t num_rows) { - return num_rows * metadata_.fixed_length + padding_for_vectors; -} +void RowTableEncoder::PrepareKeyColumnArrays(int64_t start_row, int64_t num_rows, + const std::vector& cols_in) { + const auto num_cols = static_cast(cols_in.size()); + DCHECK(batch_all_cols_.size() == num_cols); -int64_t KeyEncoder::KeyRowArray::size_rows_varying_length(int64_t num_bytes) { - return num_bytes + padding_for_vectors; -} + uint32_t num_varbinary_visited = 0; + for (uint32_t i = 0; i < num_cols; ++i) { + const KeyColumnArray& col = cols_in[row_metadata_.column_order[i]]; + KeyColumnArray col_window = col.Slice(start_row, num_rows); -void KeyEncoder::KeyRowArray::update_buffer_pointers() { - buffers_[0] = mutable_buffers_[0] = null_masks_->mutable_data(); - if (metadata_.is_fixed_length) { - buffers_[1] = mutable_buffers_[1] = rows_->mutable_data(); - buffers_[2] = mutable_buffers_[2] = nullptr; - } else { - buffers_[1] = mutable_buffers_[1] = offsets_->mutable_data(); - buffers_[2] = mutable_buffers_[2] = rows_->mutable_data(); + batch_all_cols_[i] = col_window; + if (!col.metadata().is_fixed_length) { + DCHECK(num_varbinary_visited < batch_varbinary_cols_.size()); + // If start row is zero, then base offset of varbinary column is also zero. + if (start_row == 0) { + batch_varbinary_cols_base_offsets_[num_varbinary_visited] = 0; + } else { + batch_varbinary_cols_base_offsets_[num_varbinary_visited] = + col.offsets()[start_row]; + } + batch_varbinary_cols_[num_varbinary_visited++] = col_window; + } } } -Status KeyEncoder::KeyRowArray::ResizeFixedLengthBuffers(int64_t num_extra_rows) { - if (rows_capacity_ >= num_rows_ + num_extra_rows) { - return Status::OK(); - } - - int64_t rows_capacity_new = std::max(static_cast(1), 2 * rows_capacity_); - while (rows_capacity_new < num_rows_ + num_extra_rows) { - rows_capacity_new *= 2; - } +void RowTableEncoder::DecodeFixedLengthBuffers(int64_t start_row_input, + int64_t start_row_output, int64_t num_rows, + const RowTableImpl& rows, + std::vector* cols) { + // Prepare column array vectors + PrepareKeyColumnArrays(start_row_output, num_rows, *cols); - // Null masks - RETURN_NOT_OK(null_masks_->Resize(size_null_masks(rows_capacity_new), false)); - memset(null_masks_->mutable_data() + size_null_masks(rows_capacity_), 0, - size_null_masks(rows_capacity_new) - size_null_masks(rows_capacity_)); + // Create two temp vectors with 16-bit elements + auto temp_buffer_holder_A = + util::TempVectorHolder(ctx_->stack, static_cast(num_rows)); + auto temp_buffer_A = KeyColumnArray( + KeyColumnMetadata(true, sizeof(uint16_t)), num_rows, nullptr, + reinterpret_cast(temp_buffer_holder_A.mutable_data()), nullptr); + auto temp_buffer_holder_B = + util::TempVectorHolder(ctx_->stack, static_cast(num_rows)); + auto temp_buffer_B = KeyColumnArray( + KeyColumnMetadata(true, sizeof(uint16_t)), num_rows, nullptr, + reinterpret_cast(temp_buffer_holder_B.mutable_data()), nullptr); - // Either offsets or rows - if (!metadata_.is_fixed_length) { - RETURN_NOT_OK(offsets_->Resize(size_offsets(rows_capacity_new), false)); - memset(offsets_->mutable_data() + size_offsets(rows_capacity_), 0, - size_offsets(rows_capacity_new) - size_offsets(rows_capacity_)); - } else { - RETURN_NOT_OK(rows_->Resize(size_rows_fixed_length(rows_capacity_new), false)); - memset(rows_->mutable_data() + size_rows_fixed_length(rows_capacity_), 0, - size_rows_fixed_length(rows_capacity_new) - - size_rows_fixed_length(rows_capacity_)); - bytes_capacity_ = size_rows_fixed_length(rows_capacity_new) - padding_for_vectors; + bool is_row_fixed_length = row_metadata_.is_fixed_length; + if (!is_row_fixed_length) { + EncoderOffsets::Decode(static_cast(start_row_input), + static_cast(num_rows), rows, &batch_varbinary_cols_, + batch_varbinary_cols_base_offsets_, ctx_); } - update_buffer_pointers(); - - rows_capacity_ = rows_capacity_new; + // Process fixed length columns + const auto num_cols = static_cast(batch_all_cols_.size()); + for (uint32_t i = 0; i < num_cols;) { + if (!batch_all_cols_[i].metadata().is_fixed_length || + batch_all_cols_[i].metadata().is_null_type) { + i += 1; + continue; + } + bool can_process_pair = + (i + 1 < num_cols) && batch_all_cols_[i + 1].metadata().is_fixed_length && + EncoderBinaryPair::CanProcessPair(batch_all_cols_[i].metadata(), + batch_all_cols_[i + 1].metadata()); + if (!can_process_pair) { + EncoderBinary::Decode(static_cast(start_row_input), + static_cast(num_rows), + row_metadata_.column_offsets[i], rows, &batch_all_cols_[i], + ctx_, &temp_buffer_A); + i += 1; + } else { + EncoderBinaryPair::Decode( + static_cast(start_row_input), static_cast(num_rows), + row_metadata_.column_offsets[i], rows, &batch_all_cols_[i], + &batch_all_cols_[i + 1], ctx_, &temp_buffer_A, &temp_buffer_B); + i += 2; + } + } - return Status::OK(); + // Process nulls + EncoderNulls::Decode(static_cast(start_row_input), + static_cast(num_rows), rows, &batch_all_cols_); } -Status KeyEncoder::KeyRowArray::ResizeOptionalVaryingLengthBuffer( - int64_t num_extra_bytes) { - int64_t num_bytes = offsets()[num_rows_]; - if (bytes_capacity_ >= num_bytes + num_extra_bytes || metadata_.is_fixed_length) { - return Status::OK(); - } +void RowTableEncoder::DecodeVaryingLengthBuffers(int64_t start_row_input, + int64_t start_row_output, + int64_t num_rows, + const RowTableImpl& rows, + std::vector* cols) { + // Prepare column array vectors + PrepareKeyColumnArrays(start_row_output, num_rows, *cols); - int64_t bytes_capacity_new = std::max(static_cast(1), 2 * bytes_capacity_); - while (bytes_capacity_new < num_bytes + num_extra_bytes) { - bytes_capacity_new *= 2; + bool is_row_fixed_length = row_metadata_.is_fixed_length; + if (!is_row_fixed_length) { + for (size_t i = 0; i < batch_varbinary_cols_.size(); ++i) { + // Memcpy varbinary fields into precomputed in the previous step + // positions in the output row buffer. + EncoderVarBinary::Decode(static_cast(start_row_input), + static_cast(num_rows), static_cast(i), + rows, &batch_varbinary_cols_[i], ctx_); + } } +} - RETURN_NOT_OK(rows_->Resize(size_rows_varying_length(bytes_capacity_new), false)); - memset(rows_->mutable_data() + size_rows_varying_length(bytes_capacity_), 0, - size_rows_varying_length(bytes_capacity_new) - - size_rows_varying_length(bytes_capacity_)); - - update_buffer_pointers(); +void RowTableEncoder::PrepareEncodeSelected(int64_t start_row, int64_t num_rows, + const std::vector& cols) { + // Prepare column array vectors + PrepareKeyColumnArrays(start_row, num_rows, cols); +} - bytes_capacity_ = bytes_capacity_new; +Status RowTableEncoder::EncodeSelected(RowTableImpl* rows, uint32_t num_selected, + const uint16_t* selection) { + rows->Clean(); + RETURN_NOT_OK( + rows->AppendEmpty(static_cast(num_selected), static_cast(0))); - return Status::OK(); -} + EncoderOffsets::GetRowOffsetsSelected(rows, batch_varbinary_cols_, num_selected, + selection); -Status KeyEncoder::KeyRowArray::AppendSelectionFrom(const KeyRowArray& from, - uint32_t num_rows_to_append, - const uint16_t* source_row_ids) { - DCHECK(metadata_.is_compatible(from.metadata())); - - RETURN_NOT_OK(ResizeFixedLengthBuffers(num_rows_to_append)); - - if (!metadata_.is_fixed_length) { - // Varying-length rows - auto from_offsets = reinterpret_cast(from.offsets_->data()); - auto to_offsets = reinterpret_cast(offsets_->mutable_data()); - uint32_t total_length = to_offsets[num_rows_]; - uint32_t total_length_to_append = 0; - for (uint32_t i = 0; i < num_rows_to_append; ++i) { - uint16_t row_id = source_row_ids ? source_row_ids[i] : i; - uint32_t length = from_offsets[row_id + 1] - from_offsets[row_id]; - total_length_to_append += length; - to_offsets[num_rows_ + i + 1] = total_length + total_length_to_append; - } + RETURN_NOT_OK(rows->AppendEmpty(static_cast(0), + static_cast(rows->offsets()[num_selected]))); - RETURN_NOT_OK(ResizeOptionalVaryingLengthBuffer(total_length_to_append)); - - const uint8_t* src = from.rows_->data(); - uint8_t* dst = rows_->mutable_data() + total_length; - for (uint32_t i = 0; i < num_rows_to_append; ++i) { - uint16_t row_id = source_row_ids ? source_row_ids[i] : i; - uint32_t length = from_offsets[row_id + 1] - from_offsets[row_id]; - auto src64 = reinterpret_cast(src + from_offsets[row_id]); - auto dst64 = reinterpret_cast(dst); - for (uint32_t j = 0; j < bit_util::CeilDiv(length, 8); ++j) { - dst64[j] = src64[j]; - } - dst += length; - } - } else { - // Fixed-length rows - const uint8_t* src = from.rows_->data(); - uint8_t* dst = rows_->mutable_data() + num_rows_ * metadata_.fixed_length; - for (uint32_t i = 0; i < num_rows_to_append; ++i) { - uint16_t row_id = source_row_ids ? source_row_ids[i] : i; - uint32_t length = metadata_.fixed_length; - auto src64 = reinterpret_cast(src + length * row_id); - auto dst64 = reinterpret_cast(dst); - for (uint32_t j = 0; j < bit_util::CeilDiv(length, 8); ++j) { - dst64[j] = src64[j]; - } - dst += length; + for (size_t icol = 0; icol < batch_all_cols_.size(); ++icol) { + if (batch_all_cols_[icol].metadata().is_fixed_length) { + uint32_t offset_within_row = rows->metadata().column_offsets[icol]; + EncoderBinary::EncodeSelected(offset_within_row, rows, batch_all_cols_[icol], + num_selected, selection); } } - // Null masks - uint32_t byte_length = metadata_.null_masks_bytes_per_row; - uint64_t dst_byte_offset = num_rows_ * byte_length; - const uint8_t* src_base = from.null_masks_->data(); - uint8_t* dst_base = null_masks_->mutable_data(); - for (uint32_t i = 0; i < num_rows_to_append; ++i) { - uint32_t row_id = source_row_ids ? source_row_ids[i] : i; - int64_t src_byte_offset = row_id * byte_length; - const uint8_t* src = src_base + src_byte_offset; - uint8_t* dst = dst_base + dst_byte_offset; - for (uint32_t ibyte = 0; ibyte < byte_length; ++ibyte) { - dst[ibyte] = src[ibyte]; - } - dst_byte_offset += byte_length; + EncoderOffsets::EncodeSelected(rows, batch_varbinary_cols_, num_selected, selection); + + for (size_t icol = 0; icol < batch_varbinary_cols_.size(); ++icol) { + EncoderVarBinary::EncodeSelected(static_cast(icol), rows, + batch_varbinary_cols_[icol], num_selected, + selection); } - num_rows_ += num_rows_to_append; + EncoderNulls::EncodeSelected(rows, batch_all_cols_, num_selected, selection); return Status::OK(); } -Status KeyEncoder::KeyRowArray::AppendEmpty(uint32_t num_rows_to_append, - uint32_t num_extra_bytes_to_append) { - RETURN_NOT_OK(ResizeFixedLengthBuffers(num_rows_to_append)); - RETURN_NOT_OK(ResizeOptionalVaryingLengthBuffer(num_extra_bytes_to_append)); - num_rows_ += num_rows_to_append; - if (metadata_.row_alignment > 1 || metadata_.string_alignment > 1) { - memset(rows_->mutable_data(), 0, bytes_capacity_); +namespace { +struct TransformBoolean { + static KeyColumnArray ArrayReplace(const KeyColumnArray& column, + const KeyColumnArray& temp) { + // Make sure that the temp buffer is large enough + DCHECK(temp.length() >= column.length() && temp.metadata().is_fixed_length && + temp.metadata().fixed_length >= sizeof(uint8_t)); + KeyColumnMetadata metadata; + metadata.is_fixed_length = true; + metadata.fixed_length = sizeof(uint8_t); + constexpr int buffer_index = 1; + return column.WithBufferFrom(temp, buffer_index).WithMetadata(metadata); } - return Status::OK(); -} -bool KeyEncoder::KeyRowArray::has_any_nulls(const KeyEncoderContext* ctx) const { - if (has_any_nulls_) { - return true; - } - if (num_rows_for_has_any_nulls_ < num_rows_) { - auto size_per_row = metadata().null_masks_bytes_per_row; - has_any_nulls_ = !util::bit_util::are_all_bytes_zero( - ctx->hardware_flags, null_masks() + size_per_row * num_rows_for_has_any_nulls_, - static_cast(size_per_row * (num_rows_ - num_rows_for_has_any_nulls_))); - num_rows_for_has_any_nulls_ = num_rows_; - } - return has_any_nulls_; -} + static void PostDecode(const KeyColumnArray& input, KeyColumnArray* output, + LightContext* ctx) { + // Make sure that metadata and lengths are compatible. + DCHECK(output->metadata().is_fixed_length == input.metadata().is_fixed_length); + DCHECK(output->metadata().fixed_length == 0 && input.metadata().fixed_length == 1); + DCHECK(output->length() == input.length()); + constexpr int buffer_index = 1; + DCHECK(input.data(buffer_index) != nullptr); + DCHECK(output->mutable_data(buffer_index) != nullptr); -KeyColumnArray KeyEncoder::TransformBoolean::ArrayReplace(const KeyColumnArray& column, - const KeyColumnArray& temp) { - // Make sure that the temp buffer is large enough - DCHECK(temp.length() >= column.length() && temp.metadata().is_fixed_length && - temp.metadata().fixed_length >= sizeof(uint8_t)); - KeyColumnMetadata metadata; - metadata.is_fixed_length = true; - metadata.fixed_length = sizeof(uint8_t); - constexpr int buffer_index = 1; - return column.WithBufferFrom(temp, buffer_index).WithMetadata(metadata); -} + util::bit_util::bytes_to_bits( + ctx->hardware_flags, static_cast(input.length()), input.data(buffer_index), + output->mutable_data(buffer_index), output->bit_offset(buffer_index)); + } +}; -void KeyEncoder::TransformBoolean::PostDecode(const KeyColumnArray& input, - KeyColumnArray* output, - KeyEncoderContext* ctx) { - // Make sure that metadata and lengths are compatible. - DCHECK(output->metadata().is_fixed_length == input.metadata().is_fixed_length); - DCHECK(output->metadata().fixed_length == 0 && input.metadata().fixed_length == 1); - DCHECK(output->length() == input.length()); - constexpr int buffer_index = 1; - DCHECK(input.data(buffer_index) != nullptr); - DCHECK(output->mutable_data(buffer_index) != nullptr); - - util::bit_util::bytes_to_bits( - ctx->hardware_flags, static_cast(input.length()), input.data(buffer_index), - output->mutable_data(buffer_index), output->bit_offset(buffer_index)); -} +} // namespace -bool KeyEncoder::EncoderInteger::IsBoolean(const KeyColumnMetadata& metadata) { +bool EncoderInteger::IsBoolean(const KeyColumnMetadata& metadata) { return metadata.is_fixed_length && metadata.fixed_length == 0 && !metadata.is_null_type; } -bool KeyEncoder::EncoderInteger::UsesTransform(const KeyColumnArray& column) { +bool EncoderInteger::UsesTransform(const KeyColumnArray& column) { return IsBoolean(column.metadata()); } -KeyColumnArray KeyEncoder::EncoderInteger::ArrayReplace(const KeyColumnArray& column, - const KeyColumnArray& temp) { +KeyColumnArray EncoderInteger::ArrayReplace(const KeyColumnArray& column, + const KeyColumnArray& temp) { if (IsBoolean(column.metadata())) { return TransformBoolean::ArrayReplace(column, temp); } return column; } -void KeyEncoder::EncoderInteger::PostDecode(const KeyColumnArray& input, - KeyColumnArray* output, - KeyEncoderContext* ctx) { +void EncoderInteger::PostDecode(const KeyColumnArray& input, KeyColumnArray* output, + LightContext* ctx) { if (IsBoolean(output->metadata())) { TransformBoolean::PostDecode(input, output, ctx); } } -void KeyEncoder::EncoderInteger::Decode(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col, - KeyEncoderContext* ctx, KeyColumnArray* temp) { +void EncoderInteger::Decode(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col, LightContext* ctx, + KeyColumnArray* temp) { KeyColumnArray col_prep; if (UsesTransform(*col)) { col_prep = ArrayReplace(*col, *temp); @@ -412,7 +317,111 @@ void KeyEncoder::EncoderInteger::Decode(uint32_t start_row, uint32_t num_rows, } } -bool KeyEncoder::EncoderBinary::IsInteger(const KeyColumnMetadata& metadata) { +template +void EncoderBinary::EncodeSelectedImp(uint32_t offset_within_row, RowTableImpl* rows, + const KeyColumnArray& col, uint32_t num_selected, + const uint16_t* selection, COPY_FN copy_fn, + SET_NULL_FN set_null_fn) { + bool is_fixed_length = rows->metadata().is_fixed_length; + if (is_fixed_length) { + uint32_t row_width = rows->metadata().fixed_length; + const uint8_t* src_base = col.data(1); + uint8_t* dst = rows->mutable_data(1) + offset_within_row; + for (uint32_t i = 0; i < num_selected; ++i) { + copy_fn(dst, src_base, selection[i]); + dst += row_width; + } + if (col.data(0)) { + const uint8_t* non_null_bits = col.data(0); + uint8_t* dst = rows->mutable_data(1) + offset_within_row; + for (uint32_t i = 0; i < num_selected; ++i) { + bool is_null = !bit_util::GetBit(non_null_bits, selection[i] + col.bit_offset(0)); + if (is_null) { + set_null_fn(dst); + } + dst += row_width; + } + } + } else { + const uint8_t* src_base = col.data(1); + uint8_t* dst = rows->mutable_data(2) + offset_within_row; + const uint32_t* offsets = rows->offsets(); + for (uint32_t i = 0; i < num_selected; ++i) { + copy_fn(dst + offsets[i], src_base, selection[i]); + } + if (col.data(0)) { + const uint8_t* non_null_bits = col.data(0); + uint8_t* dst = rows->mutable_data(2) + offset_within_row; + const uint32_t* offsets = rows->offsets(); + for (uint32_t i = 0; i < num_selected; ++i) { + bool is_null = !bit_util::GetBit(non_null_bits, selection[i] + col.bit_offset(0)); + if (is_null) { + set_null_fn(dst + offsets[i]); + } + } + } + } +} + +void EncoderBinary::EncodeSelected(uint32_t offset_within_row, RowTableImpl* rows, + const KeyColumnArray& col, uint32_t num_selected, + const uint16_t* selection) { + if (col.metadata().is_null_type) { + return; + } + uint32_t col_width = col.metadata().fixed_length; + if (col_width == 0) { + int bit_offset = col.bit_offset(1); + EncodeSelectedImp( + offset_within_row, rows, col, num_selected, selection, + [bit_offset](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { + *dst = bit_util::GetBit(src_base, irow + bit_offset) ? 0xff : 0x00; + }, + [](uint8_t* dst) { *dst = 0xae; }); + } else if (col_width == 1) { + EncodeSelectedImp( + offset_within_row, rows, col, num_selected, selection, + [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { + *dst = src_base[irow]; + }, + [](uint8_t* dst) { *dst = 0xae; }); + } else if (col_width == 2) { + EncodeSelectedImp( + offset_within_row, rows, col, num_selected, selection, + [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { + *reinterpret_cast(dst) = + reinterpret_cast(src_base)[irow]; + }, + [](uint8_t* dst) { *reinterpret_cast(dst) = 0xaeae; }); + } else if (col_width == 4) { + EncodeSelectedImp( + offset_within_row, rows, col, num_selected, selection, + [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { + *reinterpret_cast(dst) = + reinterpret_cast(src_base)[irow]; + }, + [](uint8_t* dst) { + *reinterpret_cast(dst) = static_cast(0xaeaeaeae); + }); + } else if (col_width == 8) { + EncodeSelectedImp( + offset_within_row, rows, col, num_selected, selection, + [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { + *reinterpret_cast(dst) = + reinterpret_cast(src_base)[irow]; + }, + [](uint8_t* dst) { *reinterpret_cast(dst) = 0xaeaeaeaeaeaeaeaeULL; }); + } else { + EncodeSelectedImp( + offset_within_row, rows, col, num_selected, selection, + [col_width](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { + memcpy(dst, src_base + col_width * irow, col_width); + }, + [col_width](uint8_t* dst) { memset(dst, 0xae, col_width); }); + } +} + +bool EncoderBinary::IsInteger(const KeyColumnMetadata& metadata) { if (metadata.is_null_type) { return false; } @@ -422,10 +431,9 @@ bool KeyEncoder::EncoderBinary::IsInteger(const KeyColumnMetadata& metadata) { (size == 0 || size == 1 || size == 2 || size == 4 || size == 8); } -void KeyEncoder::EncoderBinary::Decode(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col, - KeyEncoderContext* ctx, KeyColumnArray* temp) { +void EncoderBinary::Decode(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col, LightContext* ctx, KeyColumnArray* temp) { if (IsInteger(col->metadata())) { EncoderInteger::Decode(start_row, num_rows, offset_within_row, rows, col, ctx, temp); } else { @@ -460,9 +468,9 @@ void KeyEncoder::EncoderBinary::Decode(uint32_t start_row, uint32_t num_rows, } template -void KeyEncoder::EncoderBinary::DecodeImp(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col) { +void EncoderBinary::DecodeImp(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col) { DecodeHelper( start_row, num_rows, offset_within_row, &rows, nullptr, col, col, [](uint8_t* dst, const uint8_t* src, int64_t length) { @@ -474,11 +482,11 @@ void KeyEncoder::EncoderBinary::DecodeImp(uint32_t start_row, uint32_t num_rows, }); } -void KeyEncoder::EncoderBinaryPair::Decode(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col1, - KeyColumnArray* col2, KeyEncoderContext* ctx, - KeyColumnArray* temp1, KeyColumnArray* temp2) { +void EncoderBinaryPair::Decode(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col1, KeyColumnArray* col2, + LightContext* ctx, KeyColumnArray* temp1, + KeyColumnArray* temp2) { DCHECK(CanProcessPair(col1->metadata(), col2->metadata())); KeyColumnArray col_prep[2]; @@ -516,7 +524,7 @@ void KeyEncoder::EncoderBinaryPair::Decode(uint32_t start_row, uint32_t num_rows #endif if (num_processed < num_rows) { using DecodeImp_t = void (*)(uint32_t, uint32_t, uint32_t, uint32_t, - const KeyRowArray&, KeyColumnArray*, KeyColumnArray*); + const RowTableImpl&, KeyColumnArray*, KeyColumnArray*); static const DecodeImp_t DecodeImp_fn[] = { DecodeImp, DecodeImp, DecodeImp, DecodeImp, @@ -549,12 +557,10 @@ void KeyEncoder::EncoderBinaryPair::Decode(uint32_t start_row, uint32_t num_rows } template -void KeyEncoder::EncoderBinaryPair::DecodeImp(uint32_t num_rows_to_skip, - uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray& rows, - KeyColumnArray* col1, - KeyColumnArray* col2) { +void EncoderBinaryPair::DecodeImp(uint32_t num_rows_to_skip, uint32_t start_row, + uint32_t num_rows, uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col1, + KeyColumnArray* col2) { DCHECK(rows.length() >= start_row + num_rows); DCHECK(col1->length() == num_rows && col2->length() == num_rows); @@ -593,10 +599,11 @@ void KeyEncoder::EncoderBinaryPair::DecodeImp(uint32_t num_rows_to_skip, } } -void KeyEncoder::EncoderOffsets::Decode( - uint32_t start_row, uint32_t num_rows, const KeyRowArray& rows, - std::vector* varbinary_cols, - const std::vector& varbinary_cols_base_offset, KeyEncoderContext* ctx) { +void EncoderOffsets::Decode(uint32_t start_row, uint32_t num_rows, + const RowTableImpl& rows, + std::vector* varbinary_cols, + const std::vector& varbinary_cols_base_offset, + LightContext* ctx) { DCHECK(!varbinary_cols->empty()); DCHECK(varbinary_cols->size() == varbinary_cols_base_offset.size()); @@ -635,7 +642,7 @@ void KeyEncoder::EncoderOffsets::Decode( uint32_t offset_within_row = rows.metadata().fixed_length; for (size_t col = 0; col < varbinary_cols->size(); ++col) { offset_within_row += - KeyRowMetadata::padding_for_alignment(offset_within_row, string_alignment); + RowTableMetadata::padding_for_alignment(offset_within_row, string_alignment); uint32_t length = varbinary_ends[col] - offset_within_row; offset_within_row = varbinary_ends[col]; uint32_t* col_offsets = (*varbinary_cols)[col].mutable_offsets(); @@ -644,440 +651,10 @@ void KeyEncoder::EncoderOffsets::Decode( } } -void KeyEncoder::EncoderVarBinary::Decode(uint32_t start_row, uint32_t num_rows, - uint32_t varbinary_col_id, - const KeyRowArray& rows, KeyColumnArray* col, - KeyEncoderContext* ctx) { - // Output column varbinary buffer needs an extra 32B - // at the end in avx2 version and 8B otherwise. -#if defined(ARROW_HAVE_AVX2) - if (ctx->has_avx2()) { - DecodeHelper_avx2(start_row, num_rows, varbinary_col_id, rows, col); - } else { -#endif - if (varbinary_col_id == 0) { - DecodeImp(start_row, num_rows, varbinary_col_id, rows, col); - } else { - DecodeImp(start_row, num_rows, varbinary_col_id, rows, col); - } -#if defined(ARROW_HAVE_AVX2) - } -#endif -} - -template -void KeyEncoder::EncoderVarBinary::DecodeImp(uint32_t start_row, uint32_t num_rows, - uint32_t varbinary_col_id, - const KeyRowArray& rows, - KeyColumnArray* col) { - DecodeHelper( - start_row, num_rows, varbinary_col_id, &rows, nullptr, col, col, - [](uint8_t* dst, const uint8_t* src, int64_t length) { - for (uint32_t istripe = 0; istripe < bit_util::CeilDiv(length, 8); ++istripe) { - auto dst64 = reinterpret_cast(dst); - auto src64 = reinterpret_cast(src); - util::SafeStore(dst64 + istripe, src64[istripe]); - } - }); -} - -void KeyEncoder::EncoderNulls::Decode(uint32_t start_row, uint32_t num_rows, - const KeyRowArray& rows, - std::vector* cols) { - // Every output column needs to have a space for exactly the required number - // of rows. It also needs to have non-nulls bit-vector allocated and mutable. - DCHECK_GT(cols->size(), 0); - for (auto& col : *cols) { - DCHECK(col.length() == num_rows); - DCHECK(col.mutable_data(0) || col.metadata().is_null_type); - } - - const uint8_t* null_masks = rows.null_masks(); - uint32_t null_masks_bytes_per_row = rows.metadata().null_masks_bytes_per_row; - for (size_t col = 0; col < cols->size(); ++col) { - if ((*cols)[col].metadata().is_null_type) { - continue; - } - uint8_t* non_nulls = (*cols)[col].mutable_data(0); - const int bit_offset = (*cols)[col].bit_offset(0); - DCHECK_LT(bit_offset, 8); - non_nulls[0] |= 0xff << (bit_offset); - if (bit_offset + num_rows > 8) { - int bits_in_first_byte = 8 - bit_offset; - memset(non_nulls + 1, 0xff, bit_util::BytesForBits(num_rows - bits_in_first_byte)); - } - for (uint32_t row = 0; row < num_rows; ++row) { - uint32_t null_masks_bit_id = - (start_row + row) * null_masks_bytes_per_row * 8 + static_cast(col); - bool is_set = bit_util::GetBit(null_masks, null_masks_bit_id); - if (is_set) { - bit_util::ClearBit(non_nulls, bit_offset + row); - } - } - } -} - -uint32_t KeyEncoder::KeyRowMetadata::num_varbinary_cols() const { - uint32_t result = 0; - for (auto column_metadata : column_metadatas) { - if (!column_metadata.is_fixed_length) { - ++result; - } - } - return result; -} - -bool KeyEncoder::KeyRowMetadata::is_compatible(const KeyRowMetadata& other) const { - if (other.num_cols() != num_cols()) { - return false; - } - if (row_alignment != other.row_alignment || - string_alignment != other.string_alignment) { - return false; - } - for (size_t i = 0; i < column_metadatas.size(); ++i) { - if (column_metadatas[i].is_fixed_length != - other.column_metadatas[i].is_fixed_length) { - return false; - } - if (column_metadatas[i].fixed_length != other.column_metadatas[i].fixed_length) { - return false; - } - } - return true; -} - -void KeyEncoder::KeyRowMetadata::FromColumnMetadataVector( - const std::vector& cols, int in_row_alignment, - int in_string_alignment) { - column_metadatas.resize(cols.size()); - for (size_t i = 0; i < cols.size(); ++i) { - column_metadatas[i] = cols[i]; - } - - const auto num_cols = static_cast(cols.size()); - - // Sort columns. - // - // Columns are sorted based on the size in bytes of their fixed-length part. - // For the varying-length column, the fixed-length part is the 32-bit field storing - // cumulative length of varying-length fields. - // - // The rules are: - // - // a) Boolean column, marked with fixed-length 0, is considered to have fixed-length - // part of 1 byte. - // - // b) Columns with fixed-length part being power of 2 or multiple of row - // alignment precede other columns. They are sorted in decreasing order of the size of - // their fixed-length part. - // - // c) Fixed-length columns precede varying-length columns when - // both have the same size fixed-length part. - // - column_order.resize(num_cols); - for (uint32_t i = 0; i < num_cols; ++i) { - column_order[i] = i; - } - std::sort( - column_order.begin(), column_order.end(), [&cols](uint32_t left, uint32_t right) { - bool is_left_pow2 = - !cols[left].is_fixed_length || ARROW_POPCOUNT64(cols[left].fixed_length) <= 1; - bool is_right_pow2 = !cols[right].is_fixed_length || - ARROW_POPCOUNT64(cols[right].fixed_length) <= 1; - bool is_left_fixedlen = cols[left].is_fixed_length; - bool is_right_fixedlen = cols[right].is_fixed_length; - uint32_t width_left = - cols[left].is_fixed_length ? cols[left].fixed_length : sizeof(uint32_t); - uint32_t width_right = - cols[right].is_fixed_length ? cols[right].fixed_length : sizeof(uint32_t); - if (is_left_pow2 != is_right_pow2) { - return is_left_pow2; - } - if (!is_left_pow2) { - return left < right; - } - if (width_left != width_right) { - return width_left > width_right; - } - if (is_left_fixedlen != is_right_fixedlen) { - return is_left_fixedlen; - } - return left < right; - }); - - row_alignment = in_row_alignment; - string_alignment = in_string_alignment; - varbinary_end_array_offset = 0; - - column_offsets.resize(num_cols); - uint32_t num_varbinary_cols = 0; - uint32_t offset_within_row = 0; - for (uint32_t i = 0; i < num_cols; ++i) { - const KeyColumnMetadata& col = cols[column_order[i]]; - if (col.is_fixed_length && col.fixed_length != 0 && - ARROW_POPCOUNT64(col.fixed_length) != 1) { - offset_within_row += - KeyRowMetadata::padding_for_alignment(offset_within_row, string_alignment, col); - } - column_offsets[i] = offset_within_row; - if (!col.is_fixed_length) { - if (num_varbinary_cols == 0) { - varbinary_end_array_offset = offset_within_row; - } - DCHECK(column_offsets[i] - varbinary_end_array_offset == - num_varbinary_cols * sizeof(uint32_t)); - ++num_varbinary_cols; - offset_within_row += sizeof(uint32_t); - } else { - // Boolean column is a bit-vector, which is indicated by - // setting fixed length in column metadata to zero. - // It will be stored as a byte in output row. - if (col.fixed_length == 0) { - offset_within_row += 1; - } else { - offset_within_row += col.fixed_length; - } - } - } - - is_fixed_length = (num_varbinary_cols == 0); - fixed_length = - offset_within_row + - KeyRowMetadata::padding_for_alignment( - offset_within_row, num_varbinary_cols == 0 ? row_alignment : string_alignment); - - // We set the number of bytes per row storing null masks of individual key columns - // to be a power of two. This is not required. It could be also set to the minimal - // number of bytes required for a given number of bits (one bit per column). - null_masks_bytes_per_row = 1; - while (static_cast(null_masks_bytes_per_row * 8) < num_cols) { - null_masks_bytes_per_row *= 2; - } -} - -void KeyEncoder::Init(const std::vector& cols, KeyEncoderContext* ctx, - int row_alignment, int string_alignment) { - ctx_ = ctx; - row_metadata_.FromColumnMetadataVector(cols, row_alignment, string_alignment); - uint32_t num_cols = row_metadata_.num_cols(); - uint32_t num_varbinary_cols = row_metadata_.num_varbinary_cols(); - batch_all_cols_.resize(num_cols); - batch_varbinary_cols_.resize(num_varbinary_cols); - batch_varbinary_cols_base_offsets_.resize(num_varbinary_cols); -} - -void KeyEncoder::PrepareKeyColumnArrays(int64_t start_row, int64_t num_rows, - const std::vector& cols_in) { - const auto num_cols = static_cast(cols_in.size()); - DCHECK(batch_all_cols_.size() == num_cols); - - uint32_t num_varbinary_visited = 0; - for (uint32_t i = 0; i < num_cols; ++i) { - const KeyColumnArray& col = cols_in[row_metadata_.column_order[i]]; - KeyColumnArray col_window = col.Slice(start_row, num_rows); - - batch_all_cols_[i] = col_window; - if (!col.metadata().is_fixed_length) { - DCHECK(num_varbinary_visited < batch_varbinary_cols_.size()); - // If start row is zero, then base offset of varbinary column is also zero. - if (start_row == 0) { - batch_varbinary_cols_base_offsets_[num_varbinary_visited] = 0; - } else { - batch_varbinary_cols_base_offsets_[num_varbinary_visited] = - col.offsets()[start_row]; - } - batch_varbinary_cols_[num_varbinary_visited++] = col_window; - } - } -} - -void KeyEncoder::DecodeFixedLengthBuffers(int64_t start_row_input, - int64_t start_row_output, int64_t num_rows, - const KeyRowArray& rows, - std::vector* cols) { - // Prepare column array vectors - PrepareKeyColumnArrays(start_row_output, num_rows, *cols); - - // Create two temp vectors with 16-bit elements - auto temp_buffer_holder_A = - util::TempVectorHolder(ctx_->stack, static_cast(num_rows)); - auto temp_buffer_A = KeyColumnArray( - KeyColumnMetadata(true, sizeof(uint16_t)), num_rows, nullptr, - reinterpret_cast(temp_buffer_holder_A.mutable_data()), nullptr); - auto temp_buffer_holder_B = - util::TempVectorHolder(ctx_->stack, static_cast(num_rows)); - auto temp_buffer_B = KeyColumnArray( - KeyColumnMetadata(true, sizeof(uint16_t)), num_rows, nullptr, - reinterpret_cast(temp_buffer_holder_B.mutable_data()), nullptr); - - bool is_row_fixed_length = row_metadata_.is_fixed_length; - if (!is_row_fixed_length) { - EncoderOffsets::Decode(static_cast(start_row_input), - static_cast(num_rows), rows, &batch_varbinary_cols_, - batch_varbinary_cols_base_offsets_, ctx_); - } - - // Process fixed length columns - const auto num_cols = static_cast(batch_all_cols_.size()); - for (uint32_t i = 0; i < num_cols;) { - if (!batch_all_cols_[i].metadata().is_fixed_length || - batch_all_cols_[i].metadata().is_null_type) { - i += 1; - continue; - } - bool can_process_pair = - (i + 1 < num_cols) && batch_all_cols_[i + 1].metadata().is_fixed_length && - EncoderBinaryPair::CanProcessPair(batch_all_cols_[i].metadata(), - batch_all_cols_[i + 1].metadata()); - if (!can_process_pair) { - EncoderBinary::Decode(static_cast(start_row_input), - static_cast(num_rows), - row_metadata_.column_offsets[i], rows, &batch_all_cols_[i], - ctx_, &temp_buffer_A); - i += 1; - } else { - EncoderBinaryPair::Decode( - static_cast(start_row_input), static_cast(num_rows), - row_metadata_.column_offsets[i], rows, &batch_all_cols_[i], - &batch_all_cols_[i + 1], ctx_, &temp_buffer_A, &temp_buffer_B); - i += 2; - } - } - - // Process nulls - EncoderNulls::Decode(static_cast(start_row_input), - static_cast(num_rows), rows, &batch_all_cols_); -} - -void KeyEncoder::DecodeVaryingLengthBuffers(int64_t start_row_input, - int64_t start_row_output, int64_t num_rows, - const KeyRowArray& rows, - std::vector* cols) { - // Prepare column array vectors - PrepareKeyColumnArrays(start_row_output, num_rows, *cols); - - bool is_row_fixed_length = row_metadata_.is_fixed_length; - if (!is_row_fixed_length) { - for (size_t i = 0; i < batch_varbinary_cols_.size(); ++i) { - // Memcpy varbinary fields into precomputed in the previous step - // positions in the output row buffer. - EncoderVarBinary::Decode(static_cast(start_row_input), - static_cast(num_rows), static_cast(i), - rows, &batch_varbinary_cols_[i], ctx_); - } - } -} - -template -void KeyEncoder::EncoderBinary::EncodeSelectedImp( - uint32_t offset_within_row, KeyRowArray* rows, const KeyColumnArray& col, - uint32_t num_selected, const uint16_t* selection, COPY_FN copy_fn, - SET_NULL_FN set_null_fn) { - bool is_fixed_length = rows->metadata().is_fixed_length; - if (is_fixed_length) { - uint32_t row_width = rows->metadata().fixed_length; - const uint8_t* src_base = col.data(1); - uint8_t* dst = rows->mutable_data(1) + offset_within_row; - for (uint32_t i = 0; i < num_selected; ++i) { - copy_fn(dst, src_base, selection[i]); - dst += row_width; - } - if (col.data(0)) { - const uint8_t* non_null_bits = col.data(0); - uint8_t* dst = rows->mutable_data(1) + offset_within_row; - for (uint32_t i = 0; i < num_selected; ++i) { - bool is_null = !bit_util::GetBit(non_null_bits, selection[i] + col.bit_offset(0)); - if (is_null) { - set_null_fn(dst); - } - dst += row_width; - } - } - } else { - const uint8_t* src_base = col.data(1); - uint8_t* dst = rows->mutable_data(2) + offset_within_row; - const uint32_t* offsets = rows->offsets(); - for (uint32_t i = 0; i < num_selected; ++i) { - copy_fn(dst + offsets[i], src_base, selection[i]); - } - if (col.data(0)) { - const uint8_t* non_null_bits = col.data(0); - uint8_t* dst = rows->mutable_data(2) + offset_within_row; - const uint32_t* offsets = rows->offsets(); - for (uint32_t i = 0; i < num_selected; ++i) { - bool is_null = !bit_util::GetBit(non_null_bits, selection[i] + col.bit_offset(0)); - if (is_null) { - set_null_fn(dst + offsets[i]); - } - } - } - } -} - -void KeyEncoder::EncoderBinary::EncodeSelected(uint32_t offset_within_row, - KeyRowArray* rows, - const KeyColumnArray& col, - uint32_t num_selected, - const uint16_t* selection) { - if (col.metadata().is_null_type) { - return; - } - uint32_t col_width = col.metadata().fixed_length; - if (col_width == 0) { - int bit_offset = col.bit_offset(1); - EncodeSelectedImp( - offset_within_row, rows, col, num_selected, selection, - [bit_offset](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { - *dst = bit_util::GetBit(src_base, irow + bit_offset) ? 0xff : 0x00; - }, - [](uint8_t* dst) { *dst = 0xae; }); - } else if (col_width == 1) { - EncodeSelectedImp( - offset_within_row, rows, col, num_selected, selection, - [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { - *dst = src_base[irow]; - }, - [](uint8_t* dst) { *dst = 0xae; }); - } else if (col_width == 2) { - EncodeSelectedImp( - offset_within_row, rows, col, num_selected, selection, - [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { - *reinterpret_cast(dst) = - reinterpret_cast(src_base)[irow]; - }, - [](uint8_t* dst) { *reinterpret_cast(dst) = 0xaeae; }); - } else if (col_width == 4) { - EncodeSelectedImp( - offset_within_row, rows, col, num_selected, selection, - [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { - *reinterpret_cast(dst) = - reinterpret_cast(src_base)[irow]; - }, - [](uint8_t* dst) { - *reinterpret_cast(dst) = static_cast(0xaeaeaeae); - }); - } else if (col_width == 8) { - EncodeSelectedImp( - offset_within_row, rows, col, num_selected, selection, - [](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { - *reinterpret_cast(dst) = - reinterpret_cast(src_base)[irow]; - }, - [](uint8_t* dst) { *reinterpret_cast(dst) = 0xaeaeaeaeaeaeaeaeULL; }); - } else { - EncodeSelectedImp( - offset_within_row, rows, col, num_selected, selection, - [col_width](uint8_t* dst, const uint8_t* src_base, uint16_t irow) { - memcpy(dst, src_base + col_width * irow, col_width); - }, - [col_width](uint8_t* dst) { memset(dst, 0xae, col_width); }); - } -} - -void KeyEncoder::EncoderOffsets::GetRowOffsetsSelected( - KeyRowArray* rows, const std::vector& cols, uint32_t num_selected, - const uint16_t* selection) { +void EncoderOffsets::GetRowOffsetsSelected(RowTableImpl* rows, + const std::vector& cols, + uint32_t num_selected, + const uint16_t* selection) { if (rows->metadata().is_fixed_length) { return; } @@ -1094,7 +671,7 @@ void KeyEncoder::EncoderOffsets::GetRowOffsetsSelected( for (uint32_t i = 0; i < num_selected; ++i) { uint32_t irow = selection[i]; uint32_t length = col_offsets[irow + 1] - col_offsets[irow]; - row_offsets[i] += KeyRowMetadata::padding_for_alignment( + row_offsets[i] += RowTableMetadata::padding_for_alignment( row_offsets[i], rows->metadata().string_alignment); row_offsets[i] += length; } @@ -1118,7 +695,7 @@ void KeyEncoder::EncoderOffsets::GetRowOffsetsSelected( int row_alignment = rows->metadata().row_alignment; for (uint32_t i = 0; i < num_selected; ++i) { uint32_t length = row_offsets[i]; - length += KeyRowMetadata::padding_for_alignment(length, row_alignment); + length += RowTableMetadata::padding_for_alignment(length, row_alignment); row_offsets[i] = sum; sum += length; } @@ -1126,9 +703,9 @@ void KeyEncoder::EncoderOffsets::GetRowOffsetsSelected( } template -void KeyEncoder::EncoderOffsets::EncodeSelectedImp( - uint32_t ivarbinary, KeyRowArray* rows, const std::vector& cols, - uint32_t num_selected, const uint16_t* selection) { +void EncoderOffsets::EncodeSelectedImp(uint32_t ivarbinary, RowTableImpl* rows, + const std::vector& cols, + uint32_t num_selected, const uint16_t* selection) { const uint32_t* row_offsets = rows->offsets(); uint8_t* row_base = rows->mutable_data(2) + rows->metadata().varbinary_end_array_offset + @@ -1150,17 +727,16 @@ void KeyEncoder::EncoderOffsets::EncodeSelectedImp( row[0] = rows->metadata().fixed_length + length; } else { row[0] = row[-1] + - KeyRowMetadata::padding_for_alignment(row[-1], - rows->metadata().string_alignment) + + RowTableMetadata::padding_for_alignment( + row[-1], rows->metadata().string_alignment) + length; } } } -void KeyEncoder::EncoderOffsets::EncodeSelected(KeyRowArray* rows, - const std::vector& cols, - uint32_t num_selected, - const uint16_t* selection) { +void EncoderOffsets::EncodeSelected(RowTableImpl* rows, + const std::vector& cols, + uint32_t num_selected, const uint16_t* selection) { if (rows->metadata().is_fixed_length) { return; } @@ -1182,10 +758,79 @@ void KeyEncoder::EncoderOffsets::EncodeSelected(KeyRowArray* rows, } } -void KeyEncoder::EncoderVarBinary::EncodeSelected(uint32_t ivarbinary, KeyRowArray* rows, - const KeyColumnArray& cols, - uint32_t num_selected, - const uint16_t* selection) { +void EncoderVarBinary::Decode(uint32_t start_row, uint32_t num_rows, + uint32_t varbinary_col_id, const RowTableImpl& rows, + KeyColumnArray* col, LightContext* ctx) { + // Output column varbinary buffer needs an extra 32B + // at the end in avx2 version and 8B otherwise. +#if defined(ARROW_HAVE_AVX2) + if (ctx->has_avx2()) { + DecodeHelper_avx2(start_row, num_rows, varbinary_col_id, rows, col); + } else { +#endif + if (varbinary_col_id == 0) { + DecodeImp(start_row, num_rows, varbinary_col_id, rows, col); + } else { + DecodeImp(start_row, num_rows, varbinary_col_id, rows, col); + } +#if defined(ARROW_HAVE_AVX2) + } +#endif +} + +template +void EncoderVarBinary::DecodeImp(uint32_t start_row, uint32_t num_rows, + uint32_t varbinary_col_id, const RowTableImpl& rows, + KeyColumnArray* col) { + DecodeHelper( + start_row, num_rows, varbinary_col_id, &rows, nullptr, col, col, + [](uint8_t* dst, const uint8_t* src, int64_t length) { + for (uint32_t istripe = 0; istripe < bit_util::CeilDiv(length, 8); ++istripe) { + auto dst64 = reinterpret_cast(dst); + auto src64 = reinterpret_cast(src); + util::SafeStore(dst64 + istripe, src64[istripe]); + } + }); +} + +void EncoderNulls::Decode(uint32_t start_row, uint32_t num_rows, const RowTableImpl& rows, + std::vector* cols) { + // Every output column needs to have a space for exactly the required number + // of rows. It also needs to have non-nulls bit-vector allocated and mutable. + DCHECK_GT(cols->size(), 0); + for (auto& col : *cols) { + DCHECK(col.length() == num_rows); + DCHECK(col.mutable_data(0) || col.metadata().is_null_type); + } + + const uint8_t* null_masks = rows.null_masks(); + uint32_t null_masks_bytes_per_row = rows.metadata().null_masks_bytes_per_row; + for (size_t col = 0; col < cols->size(); ++col) { + if ((*cols)[col].metadata().is_null_type) { + continue; + } + uint8_t* non_nulls = (*cols)[col].mutable_data(0); + const int bit_offset = (*cols)[col].bit_offset(0); + DCHECK_LT(bit_offset, 8); + non_nulls[0] |= 0xff << (bit_offset); + if (bit_offset + num_rows > 8) { + int bits_in_first_byte = 8 - bit_offset; + memset(non_nulls + 1, 0xff, bit_util::BytesForBits(num_rows - bits_in_first_byte)); + } + for (uint32_t row = 0; row < num_rows; ++row) { + uint32_t null_masks_bit_id = + (start_row + row) * null_masks_bytes_per_row * 8 + static_cast(col); + bool is_set = bit_util::GetBit(null_masks, null_masks_bit_id); + if (is_set) { + bit_util::ClearBit(non_nulls, bit_offset + row); + } + } + } +} + +void EncoderVarBinary::EncodeSelected(uint32_t ivarbinary, RowTableImpl* rows, + const KeyColumnArray& cols, uint32_t num_selected, + const uint16_t* selection) { const uint32_t* row_offsets = rows->offsets(); uint8_t* row_base = rows->mutable_data(2); const uint32_t* col_offsets = cols.offsets(); @@ -1213,10 +858,9 @@ void KeyEncoder::EncoderVarBinary::EncodeSelected(uint32_t ivarbinary, KeyRowArr } } -void KeyEncoder::EncoderNulls::EncodeSelected(KeyRowArray* rows, - const std::vector& cols, - uint32_t num_selected, - const uint16_t* selection) { +void EncoderNulls::EncodeSelected(RowTableImpl* rows, + const std::vector& cols, + uint32_t num_selected, const uint16_t* selection) { uint8_t* null_masks = rows->null_masks(); uint32_t null_mask_num_bytes = rows->metadata().null_masks_bytes_per_row; memset(null_masks, 0, null_mask_num_bytes * num_selected); @@ -1234,44 +878,5 @@ void KeyEncoder::EncoderNulls::EncodeSelected(KeyRowArray* rows, } } -void KeyEncoder::PrepareEncodeSelected(int64_t start_row, int64_t num_rows, - const std::vector& cols) { - // Prepare column array vectors - PrepareKeyColumnArrays(start_row, num_rows, cols); -} - -Status KeyEncoder::EncodeSelected(KeyRowArray* rows, uint32_t num_selected, - const uint16_t* selection) { - rows->Clean(); - RETURN_NOT_OK( - rows->AppendEmpty(static_cast(num_selected), static_cast(0))); - - EncoderOffsets::GetRowOffsetsSelected(rows, batch_varbinary_cols_, num_selected, - selection); - - RETURN_NOT_OK(rows->AppendEmpty(static_cast(0), - static_cast(rows->offsets()[num_selected]))); - - for (size_t icol = 0; icol < batch_all_cols_.size(); ++icol) { - if (batch_all_cols_[icol].metadata().is_fixed_length) { - uint32_t offset_within_row = rows->metadata().column_offsets[icol]; - EncoderBinary::EncodeSelected(offset_within_row, rows, batch_all_cols_[icol], - num_selected, selection); - } - } - - EncoderOffsets::EncodeSelected(rows, batch_varbinary_cols_, num_selected, selection); - - for (size_t icol = 0; icol < batch_varbinary_cols_.size(); ++icol) { - EncoderVarBinary::EncodeSelected(static_cast(icol), rows, - batch_varbinary_cols_[icol], num_selected, - selection); - } - - EncoderNulls::EncodeSelected(rows, batch_all_cols_, num_selected, selection); - - return Status::OK(); -} - } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/row/encode_internal.h b/cpp/src/arrow/compute/row/encode_internal.h new file mode 100644 index 00000000000..ce887313466 --- /dev/null +++ b/cpp/src/arrow/compute/row/encode_internal.h @@ -0,0 +1,323 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include + +#include "arrow/array/data.h" +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/util.h" +#include "arrow/compute/light_array.h" +#include "arrow/compute/row/row_internal.h" +#include "arrow/memory_pool.h" +#include "arrow/result.h" +#include "arrow/status.h" +#include "arrow/util/bit_util.h" + +namespace arrow { +namespace compute { + +/// Converts between Arrow's typical column representation to a row-based representation +/// +/// Data is stored as a single array of rows. Each row combines data from all columns. +/// The conversion is reversible. +/// +/// Row-oriented storage is beneficial when there is a need for random access +/// of individual rows and at the same time all included columns are likely to +/// be accessed together, as in the case of hash table key. +/// +/// Does not support nested types +class RowTableEncoder { + public: + void Init(const std::vector& cols, LightContext* ctx, + int row_alignment, int string_alignment); + + const RowTableMetadata& row_metadata() { return row_metadata_; } + // GrouperFastImpl right now needs somewhat intrusive visibility into RowTableEncoder + // This could be cleaned up at some point + const std::vector& batch_all_cols() { return batch_all_cols_; } + + /// \brief Prepare to encode a collection of columns + /// \param start_row The starting row to encode + /// \param num_rows The number of rows to encode + /// \param cols The columns to encode. The order of the columns should + /// be consistent with the order used to create the RowTableMetadata + void PrepareEncodeSelected(int64_t start_row, int64_t num_rows, + const std::vector& cols); + /// \brief Encode selection of prepared rows into a row table + /// \param rows The output row table + /// \param num_selected The number of rows to encode + /// \param selection indices of the rows to encode + Status EncodeSelected(RowTableImpl* rows, uint32_t num_selected, + const uint16_t* selection); + + /// \brief Decode a window of row oriented data into a corresponding + /// window of column oriented storage. + /// \param start_row_input The starting row to decode + /// \param start_row_output An offset into the output array to write to + /// \param num_rows The number of rows to decode + /// \param rows The row table to decode from + /// \param cols The columns to decode into, should be sized appropriately + /// + /// The output buffers need to be correctly allocated and sized before + /// calling each method. For that reason decoding is split into two functions. + /// DecodeFixedLengthBuffers processes everything except for varying length + /// buffers. + /// The output can be used to find out required varying length buffers sizes + /// for the call to DecodeVaryingLengthBuffers + void DecodeFixedLengthBuffers(int64_t start_row_input, int64_t start_row_output, + int64_t num_rows, const RowTableImpl& rows, + std::vector* cols); + + /// \brief Decode the varlength columns of a row table into column storage + /// \param start_row_input The starting row to decode + /// \param start_row_output An offset into the output arrays + /// \param num_rows The number of rows to decode + /// \param rows The row table to decode from + /// \param cols The column arrays to decode into + void DecodeVaryingLengthBuffers(int64_t start_row_input, int64_t start_row_output, + int64_t num_rows, const RowTableImpl& rows, + std::vector* cols); + + private: + /// Prepare column array vectors. + /// Output column arrays represent a range of input column arrays + /// specified by starting row and number of rows. + /// Three vectors are generated: + /// - all columns + /// - fixed-length columns only + /// - varying-length columns only + void PrepareKeyColumnArrays(int64_t start_row, int64_t num_rows, + const std::vector& cols_in); + + LightContext* ctx_; + + // Data initialized once, based on data types of key columns + RowTableMetadata row_metadata_; + + // Data initialized for each input batch. + // All elements are ordered according to the order of encoded fields in a row. + std::vector batch_all_cols_; + std::vector batch_varbinary_cols_; + std::vector batch_varbinary_cols_base_offsets_; +}; + +class EncoderInteger { + public: + static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col, LightContext* ctx, + KeyColumnArray* temp); + static bool UsesTransform(const KeyColumnArray& column); + static KeyColumnArray ArrayReplace(const KeyColumnArray& column, + const KeyColumnArray& temp); + static void PostDecode(const KeyColumnArray& input, KeyColumnArray* output, + LightContext* ctx); + + private: + static bool IsBoolean(const KeyColumnMetadata& metadata); +}; + +class EncoderBinary { + public: + static void EncodeSelected(uint32_t offset_within_row, RowTableImpl* rows, + const KeyColumnArray& col, uint32_t num_selected, + const uint16_t* selection); + static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col, LightContext* ctx, + KeyColumnArray* temp); + static bool IsInteger(const KeyColumnMetadata& metadata); + + private: + template + static void EncodeSelectedImp(uint32_t offset_within_row, RowTableImpl* rows, + const KeyColumnArray& col, uint32_t num_selected, + const uint16_t* selection, COPY_FN copy_fn, + SET_NULL_FN set_null_fn); + + template + static inline void DecodeHelper(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, + const RowTableImpl* rows_const, + RowTableImpl* rows_mutable_maybe_null, + const KeyColumnArray* col_const, + KeyColumnArray* col_mutable_maybe_null, + COPY_FN copy_fn) { + ARROW_DCHECK(col_const && col_const->metadata().is_fixed_length); + uint32_t col_width = col_const->metadata().fixed_length; + + if (is_row_fixed_length) { + uint32_t row_width = rows_const->metadata().fixed_length; + for (uint32_t i = 0; i < num_rows; ++i) { + const uint8_t* src; + uint8_t* dst; + src = rows_const->data(1) + row_width * (start_row + i) + offset_within_row; + dst = col_mutable_maybe_null->mutable_data(1) + col_width * i; + copy_fn(dst, src, col_width); + } + } else { + const uint32_t* row_offsets = rows_const->offsets(); + for (uint32_t i = 0; i < num_rows; ++i) { + const uint8_t* src; + uint8_t* dst; + src = rows_const->data(2) + row_offsets[start_row + i] + offset_within_row; + dst = col_mutable_maybe_null->mutable_data(1) + col_width * i; + copy_fn(dst, src, col_width); + } + } + } + + template + static void DecodeImp(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col); +#if defined(ARROW_HAVE_AVX2) + static void DecodeHelper_avx2(bool is_row_fixed_length, uint32_t start_row, + uint32_t num_rows, uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col); + template + static void DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col); +#endif +}; + +class EncoderBinaryPair { + public: + static bool CanProcessPair(const KeyColumnMetadata& col1, + const KeyColumnMetadata& col2) { + return EncoderBinary::IsInteger(col1) && EncoderBinary::IsInteger(col2); + } + static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col1, KeyColumnArray* col2, + LightContext* ctx, KeyColumnArray* temp1, KeyColumnArray* temp2); + + private: + template + static void DecodeImp(uint32_t num_rows_to_skip, uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col1, KeyColumnArray* col2); +#if defined(ARROW_HAVE_AVX2) + static uint32_t DecodeHelper_avx2(bool is_row_fixed_length, uint32_t col_width, + uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col1, KeyColumnArray* col2); + template + static uint32_t DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col1, KeyColumnArray* col2); +#endif +}; + +class EncoderOffsets { + public: + static void GetRowOffsetsSelected(RowTableImpl* rows, + const std::vector& cols, + uint32_t num_selected, const uint16_t* selection); + static void EncodeSelected(RowTableImpl* rows, const std::vector& cols, + uint32_t num_selected, const uint16_t* selection); + + static void Decode(uint32_t start_row, uint32_t num_rows, const RowTableImpl& rows, + std::vector* varbinary_cols, + const std::vector& varbinary_cols_base_offset, + LightContext* ctx); + + private: + template + static void EncodeSelectedImp(uint32_t ivarbinary, RowTableImpl* rows, + const std::vector& cols, + uint32_t num_selected, const uint16_t* selection); +}; + +class EncoderVarBinary { + public: + static void EncodeSelected(uint32_t ivarbinary, RowTableImpl* rows, + const KeyColumnArray& cols, uint32_t num_selected, + const uint16_t* selection); + + static void Decode(uint32_t start_row, uint32_t num_rows, uint32_t varbinary_col_id, + const RowTableImpl& rows, KeyColumnArray* col, LightContext* ctx); + + private: + template + static inline void DecodeHelper(uint32_t start_row, uint32_t num_rows, + uint32_t varbinary_col_id, + const RowTableImpl* rows_const, + RowTableImpl* rows_mutable_maybe_null, + const KeyColumnArray* col_const, + KeyColumnArray* col_mutable_maybe_null, + COPY_FN copy_fn) { + // Column and rows need to be varying length + ARROW_DCHECK(!rows_const->metadata().is_fixed_length && + !col_const->metadata().is_fixed_length); + + const uint32_t* row_offsets_for_batch = rows_const->offsets() + start_row; + const uint32_t* col_offsets = col_const->offsets(); + + uint32_t col_offset_next = col_offsets[0]; + for (uint32_t i = 0; i < num_rows; ++i) { + uint32_t col_offset = col_offset_next; + col_offset_next = col_offsets[i + 1]; + + uint32_t row_offset = row_offsets_for_batch[i]; + const uint8_t* row = rows_const->data(2) + row_offset; + + uint32_t offset_within_row; + uint32_t length; + if (first_varbinary_col) { + rows_const->metadata().first_varbinary_offset_and_length(row, &offset_within_row, + &length); + } else { + rows_const->metadata().nth_varbinary_offset_and_length( + row, varbinary_col_id, &offset_within_row, &length); + } + + row_offset += offset_within_row; + + const uint8_t* src; + uint8_t* dst; + src = rows_const->data(2) + row_offset; + dst = col_mutable_maybe_null->mutable_data(2) + col_offset; + copy_fn(dst, src, length); + } + } + template + static void DecodeImp(uint32_t start_row, uint32_t num_rows, uint32_t varbinary_col_id, + const RowTableImpl& rows, KeyColumnArray* col); +#if defined(ARROW_HAVE_AVX2) + static void DecodeHelper_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t varbinary_col_id, const RowTableImpl& rows, + KeyColumnArray* col); + template + static void DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t varbinary_col_id, const RowTableImpl& rows, + KeyColumnArray* col); +#endif +}; + +class EncoderNulls { + public: + static void EncodeSelected(RowTableImpl* rows, const std::vector& cols, + uint32_t num_selected, const uint16_t* selection); + + static void Decode(uint32_t start_row, uint32_t num_rows, const RowTableImpl& rows, + std::vector* cols); +}; + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/exec/key_encode_avx2.cc b/cpp/src/arrow/compute/row/encode_internal_avx2.cc similarity index 83% rename from cpp/src/arrow/compute/exec/key_encode_avx2.cc rename to cpp/src/arrow/compute/row/encode_internal_avx2.cc index 832bb0361d8..02ba310bded 100644 --- a/cpp/src/arrow/compute/exec/key_encode_avx2.cc +++ b/cpp/src/arrow/compute/row/encode_internal_avx2.cc @@ -17,18 +17,16 @@ #include -#include "arrow/compute/exec/key_encode.h" +#include "arrow/compute/row/encode_internal.h" namespace arrow { namespace compute { #if defined(ARROW_HAVE_AVX2) -void KeyEncoder::EncoderBinary::DecodeHelper_avx2(bool is_row_fixed_length, - uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray& rows, - KeyColumnArray* col) { +void EncoderBinary::DecodeHelper_avx2(bool is_row_fixed_length, uint32_t start_row, + uint32_t num_rows, uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col) { if (is_row_fixed_length) { DecodeImp_avx2(start_row, num_rows, offset_within_row, rows, col); } else { @@ -37,10 +35,9 @@ void KeyEncoder::EncoderBinary::DecodeHelper_avx2(bool is_row_fixed_length, } template -void KeyEncoder::EncoderBinary::DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, - const KeyRowArray& rows, - KeyColumnArray* col) { +void EncoderBinary::DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, const RowTableImpl& rows, + KeyColumnArray* col) { DecodeHelper( start_row, num_rows, offset_within_row, &rows, nullptr, col, col, [](uint8_t* dst, const uint8_t* src, int64_t length) { @@ -52,13 +49,13 @@ void KeyEncoder::EncoderBinary::DecodeImp_avx2(uint32_t start_row, uint32_t num_ }); } -uint32_t KeyEncoder::EncoderBinaryPair::DecodeHelper_avx2( +uint32_t EncoderBinaryPair::DecodeHelper_avx2( bool is_row_fixed_length, uint32_t col_width, uint32_t start_row, uint32_t num_rows, - uint32_t offset_within_row, const KeyRowArray& rows, KeyColumnArray* col1, + uint32_t offset_within_row, const RowTableImpl& rows, KeyColumnArray* col1, KeyColumnArray* col2) { using DecodeImp_avx2_t = uint32_t (*)(uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col1, KeyColumnArray* col2); + const RowTableImpl& rows, KeyColumnArray* col1, KeyColumnArray* col2); static const DecodeImp_avx2_t DecodeImp_avx2_fn[] = { DecodeImp_avx2, DecodeImp_avx2, DecodeImp_avx2, DecodeImp_avx2, DecodeImp_avx2, DecodeImp_avx2, @@ -70,9 +67,10 @@ uint32_t KeyEncoder::EncoderBinaryPair::DecodeHelper_avx2( } template -uint32_t KeyEncoder::EncoderBinaryPair::DecodeImp_avx2( - uint32_t start_row, uint32_t num_rows, uint32_t offset_within_row, - const KeyRowArray& rows, KeyColumnArray* col1, KeyColumnArray* col2) { +uint32_t EncoderBinaryPair::DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t offset_within_row, + const RowTableImpl& rows, KeyColumnArray* col1, + KeyColumnArray* col2) { ARROW_DCHECK(col_width == 1 || col_width == 2 || col_width == 4 || col_width == 8); uint8_t* col_vals_A = col1->mutable_data(1); @@ -207,11 +205,9 @@ uint32_t KeyEncoder::EncoderBinaryPair::DecodeImp_avx2( return num_processed; } -void KeyEncoder::EncoderVarBinary::DecodeHelper_avx2(uint32_t start_row, - uint32_t num_rows, - uint32_t varbinary_col_id, - const KeyRowArray& rows, - KeyColumnArray* col) { +void EncoderVarBinary::DecodeHelper_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t varbinary_col_id, + const RowTableImpl& rows, KeyColumnArray* col) { if (varbinary_col_id == 0) { DecodeImp_avx2(start_row, num_rows, varbinary_col_id, rows, col); } else { @@ -220,10 +216,9 @@ void KeyEncoder::EncoderVarBinary::DecodeHelper_avx2(uint32_t start_row, } template -void KeyEncoder::EncoderVarBinary::DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, - uint32_t varbinary_col_id, - const KeyRowArray& rows, - KeyColumnArray* col) { +void EncoderVarBinary::DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, + uint32_t varbinary_col_id, const RowTableImpl& rows, + KeyColumnArray* col) { DecodeHelper( start_row, num_rows, varbinary_col_id, &rows, nullptr, col, col, [](uint8_t* dst, const uint8_t* src, int64_t length) { diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc new file mode 100644 index 00000000000..eb55124b179 --- /dev/null +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -0,0 +1,590 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/row/grouper.h" + +#include + +#include "arrow/compute/exec/key_hash.h" +#include "arrow/compute/exec/key_map.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/exec_internal.h" +#include "arrow/compute/function.h" +#include "arrow/compute/kernels/row_encoder.h" +#include "arrow/compute/light_array.h" +#include "arrow/compute/registry.h" +#include "arrow/compute/row/compare_internal.h" +#include "arrow/type.h" +#include "arrow/util/bitmap_ops.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/cpu_info.h" +#include "arrow/util/logging.h" +#include "arrow/util/make_unique.h" +#include "arrow/util/task_group.h" + +namespace arrow { + +using internal::checked_cast; + +namespace compute { + +namespace { + +struct GrouperImpl : Grouper { + static Result> Make(const std::vector& keys, + ExecContext* ctx) { + auto impl = ::arrow::internal::make_unique(); + + impl->encoders_.resize(keys.size()); + impl->ctx_ = ctx; + + for (size_t i = 0; i < keys.size(); ++i) { + const auto& key = keys[i].type; + + if (key->id() == Type::BOOL) { + impl->encoders_[i] = + ::arrow::internal::make_unique(); + continue; + } + + if (key->id() == Type::DICTIONARY) { + impl->encoders_[i] = + ::arrow::internal::make_unique( + key, ctx->memory_pool()); + continue; + } + + if (is_fixed_width(key->id())) { + impl->encoders_[i] = + ::arrow::internal::make_unique(key); + continue; + } + + if (is_binary_like(key->id())) { + impl->encoders_[i] = + ::arrow::internal::make_unique>( + key); + continue; + } + + if (is_large_binary_like(key->id())) { + impl->encoders_[i] = ::arrow::internal::make_unique< + internal::VarLengthKeyEncoder>(key); + continue; + } + + if (key->id() == Type::NA) { + impl->encoders_[i] = ::arrow::internal::make_unique(); + continue; + } + + return Status::NotImplemented("Keys of type ", *key); + } + + return std::move(impl); + } + + Result Consume(const ExecBatch& batch) override { + std::vector offsets_batch(batch.length + 1); + for (int i = 0; i < batch.num_values(); ++i) { + encoders_[i]->AddLength(batch[i], batch.length, offsets_batch.data()); + } + + int32_t total_length = 0; + for (int64_t i = 0; i < batch.length; ++i) { + auto total_length_before = total_length; + total_length += offsets_batch[i]; + offsets_batch[i] = total_length_before; + } + offsets_batch[batch.length] = total_length; + + std::vector key_bytes_batch(total_length); + std::vector key_buf_ptrs(batch.length); + for (int64_t i = 0; i < batch.length; ++i) { + key_buf_ptrs[i] = key_bytes_batch.data() + offsets_batch[i]; + } + + for (int i = 0; i < batch.num_values(); ++i) { + RETURN_NOT_OK(encoders_[i]->Encode(batch[i], batch.length, key_buf_ptrs.data())); + } + + TypedBufferBuilder group_ids_batch(ctx_->memory_pool()); + RETURN_NOT_OK(group_ids_batch.Resize(batch.length)); + + for (int64_t i = 0; i < batch.length; ++i) { + int32_t key_length = offsets_batch[i + 1] - offsets_batch[i]; + std::string key( + reinterpret_cast(key_bytes_batch.data() + offsets_batch[i]), + key_length); + + auto it_success = map_.emplace(key, num_groups_); + auto group_id = it_success.first->second; + + if (it_success.second) { + // new key; update offsets and key_bytes + ++num_groups_; + // Skip if there are no keys + if (key_length > 0) { + auto next_key_offset = static_cast(key_bytes_.size()); + key_bytes_.resize(next_key_offset + key_length); + offsets_.push_back(next_key_offset + key_length); + memcpy(key_bytes_.data() + next_key_offset, key.c_str(), key_length); + } + } + + group_ids_batch.UnsafeAppend(group_id); + } + + ARROW_ASSIGN_OR_RAISE(auto group_ids, group_ids_batch.Finish()); + return Datum(UInt32Array(batch.length, std::move(group_ids))); + } + + uint32_t num_groups() const override { return num_groups_; } + + Result GetUniques() override { + ExecBatch out({}, num_groups_); + + std::vector key_buf_ptrs(num_groups_); + for (int64_t i = 0; i < num_groups_; ++i) { + key_buf_ptrs[i] = key_bytes_.data() + offsets_[i]; + } + + out.values.resize(encoders_.size()); + for (size_t i = 0; i < encoders_.size(); ++i) { + ARROW_ASSIGN_OR_RAISE( + out.values[i], + encoders_[i]->Decode(key_buf_ptrs.data(), static_cast(num_groups_), + ctx_->memory_pool())); + } + + return out; + } + + ExecContext* ctx_; + std::unordered_map map_; + std::vector offsets_ = {0}; + std::vector key_bytes_; + uint32_t num_groups_ = 0; + std::vector> encoders_; +}; + +struct GrouperFastImpl : Grouper { + static constexpr int kBitmapPaddingForSIMD = 64; // bits + static constexpr int kPaddingForSIMD = 32; // bytes + + static bool CanUse(const std::vector& keys) { +#if ARROW_LITTLE_ENDIAN + for (size_t i = 0; i < keys.size(); ++i) { + const auto& key = keys[i].type; + if (is_large_binary_like(key->id())) { + return false; + } + } + return true; +#else + return false; +#endif + } + + static Result> Make( + const std::vector& keys, ExecContext* ctx) { + auto impl = ::arrow::internal::make_unique(); + impl->ctx_ = ctx; + + RETURN_NOT_OK(impl->temp_stack_.Init(ctx->memory_pool(), 64 * minibatch_size_max_)); + impl->encode_ctx_.hardware_flags = + arrow::internal::CpuInfo::GetInstance()->hardware_flags(); + impl->encode_ctx_.stack = &impl->temp_stack_; + + auto num_columns = keys.size(); + impl->col_metadata_.resize(num_columns); + impl->key_types_.resize(num_columns); + impl->dictionaries_.resize(num_columns); + for (size_t icol = 0; icol < num_columns; ++icol) { + const auto& key = keys[icol].type; + if (key->id() == Type::DICTIONARY) { + auto bit_width = checked_cast(*key).bit_width(); + ARROW_DCHECK(bit_width % 8 == 0); + impl->col_metadata_[icol] = KeyColumnMetadata(true, bit_width / 8); + } else if (key->id() == Type::BOOL) { + impl->col_metadata_[icol] = KeyColumnMetadata(true, 0); + } else if (is_fixed_width(key->id())) { + impl->col_metadata_[icol] = KeyColumnMetadata( + true, checked_cast(*key).bit_width() / 8); + } else if (is_binary_like(key->id())) { + impl->col_metadata_[icol] = KeyColumnMetadata(false, sizeof(uint32_t)); + } else if (key->id() == Type::NA) { + impl->col_metadata_[icol] = KeyColumnMetadata(true, 0, /*is_null_type_in=*/true); + } else { + return Status::NotImplemented("Keys of type ", *key); + } + impl->key_types_[icol] = key; + } + + impl->encoder_.Init(impl->col_metadata_, &impl->encode_ctx_, + /* row_alignment = */ sizeof(uint64_t), + /* string_alignment = */ sizeof(uint64_t)); + RETURN_NOT_OK(impl->rows_.Init(ctx->memory_pool(), impl->encoder_.row_metadata())); + RETURN_NOT_OK( + impl->rows_minibatch_.Init(ctx->memory_pool(), impl->encoder_.row_metadata())); + impl->minibatch_size_ = impl->minibatch_size_min_; + GrouperFastImpl* impl_ptr = impl.get(); + auto equal_func = [impl_ptr]( + int num_keys_to_compare, const uint16_t* selection_may_be_null, + const uint32_t* group_ids, uint32_t* out_num_keys_mismatch, + uint16_t* out_selection_mismatch) { + KeyCompare::CompareColumnsToRows( + num_keys_to_compare, selection_may_be_null, group_ids, &impl_ptr->encode_ctx_, + out_num_keys_mismatch, out_selection_mismatch, + impl_ptr->encoder_.batch_all_cols(), impl_ptr->rows_); + }; + auto append_func = [impl_ptr](int num_keys, const uint16_t* selection) { + RETURN_NOT_OK(impl_ptr->encoder_.EncodeSelected(&impl_ptr->rows_minibatch_, + num_keys, selection)); + return impl_ptr->rows_.AppendSelectionFrom(impl_ptr->rows_minibatch_, num_keys, + nullptr); + }; + RETURN_NOT_OK(impl->map_.init(impl->encode_ctx_.hardware_flags, ctx->memory_pool(), + impl->encode_ctx_.stack, impl->log_minibatch_max_, + equal_func, append_func)); + impl->cols_.resize(num_columns); + impl->minibatch_hashes_.resize(impl->minibatch_size_max_ + + kPaddingForSIMD / sizeof(uint32_t)); + + return std::move(impl); + } + + ~GrouperFastImpl() { map_.cleanup(); } + + Result Consume(const ExecBatch& batch) override { + // ARROW-14027: broadcast scalar arguments for now + for (int i = 0; i < batch.num_values(); i++) { + if (batch.values[i].is_scalar()) { + ExecBatch expanded = batch; + for (int j = i; j < expanded.num_values(); j++) { + if (expanded.values[j].is_scalar()) { + ARROW_ASSIGN_OR_RAISE( + expanded.values[j], + MakeArrayFromScalar(*expanded.values[j].scalar(), expanded.length, + ctx_->memory_pool())); + } + } + return ConsumeImpl(expanded); + } + } + return ConsumeImpl(batch); + } + + Result ConsumeImpl(const ExecBatch& batch) { + int64_t num_rows = batch.length; + int num_columns = batch.num_values(); + // Process dictionaries + for (int icol = 0; icol < num_columns; ++icol) { + if (key_types_[icol]->id() == Type::DICTIONARY) { + auto data = batch[icol].array(); + auto dict = MakeArray(data->dictionary); + if (dictionaries_[icol]) { + if (!dictionaries_[icol]->Equals(dict)) { + // TODO(bkietz) unify if necessary. For now, just error if any batch's + // dictionary differs from the first we saw for this key + return Status::NotImplemented("Unifying differing dictionaries"); + } + } else { + dictionaries_[icol] = std::move(dict); + } + } + } + + std::shared_ptr group_ids; + ARROW_ASSIGN_OR_RAISE( + group_ids, AllocateBuffer(sizeof(uint32_t) * num_rows, ctx_->memory_pool())); + + for (int icol = 0; icol < num_columns; ++icol) { + const uint8_t* non_nulls = NULLPTR; + const uint8_t* fixedlen = NULLPTR; + const uint8_t* varlen = NULLPTR; + + // Skip if the key's type is NULL + if (key_types_[icol]->id() != Type::NA) { + if (batch[icol].array()->buffers[0] != NULLPTR) { + non_nulls = batch[icol].array()->buffers[0]->data(); + } + fixedlen = batch[icol].array()->buffers[1]->data(); + if (!col_metadata_[icol].is_fixed_length) { + varlen = batch[icol].array()->buffers[2]->data(); + } + } + + int64_t offset = batch[icol].array()->offset; + + auto col_base = KeyColumnArray(col_metadata_[icol], offset + num_rows, non_nulls, + fixedlen, varlen); + + cols_[icol] = col_base.Slice(offset, num_rows); + } + + // Split into smaller mini-batches + // + for (uint32_t start_row = 0; start_row < num_rows;) { + uint32_t batch_size_next = std::min(static_cast(minibatch_size_), + static_cast(num_rows) - start_row); + + // Encode + rows_minibatch_.Clean(); + encoder_.PrepareEncodeSelected(start_row, batch_size_next, cols_); + + // Compute hash + Hashing32::HashMultiColumn(encoder_.batch_all_cols(), &encode_ctx_, + minibatch_hashes_.data()); + + // Map + auto match_bitvector = + util::TempVectorHolder(&temp_stack_, (batch_size_next + 7) / 8); + { + auto local_slots = util::TempVectorHolder(&temp_stack_, batch_size_next); + map_.early_filter(batch_size_next, minibatch_hashes_.data(), + match_bitvector.mutable_data(), local_slots.mutable_data()); + map_.find(batch_size_next, minibatch_hashes_.data(), + match_bitvector.mutable_data(), local_slots.mutable_data(), + reinterpret_cast(group_ids->mutable_data()) + start_row); + } + auto ids = util::TempVectorHolder(&temp_stack_, batch_size_next); + int num_ids; + util::bit_util::bits_to_indexes(0, encode_ctx_.hardware_flags, batch_size_next, + match_bitvector.mutable_data(), &num_ids, + ids.mutable_data()); + + RETURN_NOT_OK(map_.map_new_keys( + num_ids, ids.mutable_data(), minibatch_hashes_.data(), + reinterpret_cast(group_ids->mutable_data()) + start_row)); + + start_row += batch_size_next; + + if (minibatch_size_ * 2 <= minibatch_size_max_) { + minibatch_size_ *= 2; + } + } + + return Datum(UInt32Array(batch.length, std::move(group_ids))); + } + + uint32_t num_groups() const override { return static_cast(rows_.length()); } + + // Make sure padded buffers end up with the right logical size + + Result> AllocatePaddedBitmap(int64_t length) { + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr buf, + AllocateBitmap(length + kBitmapPaddingForSIMD, ctx_->memory_pool())); + return SliceMutableBuffer(buf, 0, bit_util::BytesForBits(length)); + } + + Result> AllocatePaddedBuffer(int64_t size) { + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr buf, + AllocateBuffer(size + kBitmapPaddingForSIMD, ctx_->memory_pool())); + return SliceMutableBuffer(buf, 0, size); + } + + Result GetUniques() override { + auto num_columns = static_cast(col_metadata_.size()); + int64_t num_groups = rows_.length(); + + std::vector> non_null_bufs(num_columns); + std::vector> fixedlen_bufs(num_columns); + std::vector> varlen_bufs(num_columns); + + for (size_t i = 0; i < num_columns; ++i) { + if (col_metadata_[i].is_null_type) { + uint8_t* non_nulls = NULLPTR; + uint8_t* fixedlen = NULLPTR; + cols_[i] = + KeyColumnArray(col_metadata_[i], num_groups, non_nulls, fixedlen, NULLPTR); + continue; + } + ARROW_ASSIGN_OR_RAISE(non_null_bufs[i], AllocatePaddedBitmap(num_groups)); + if (col_metadata_[i].is_fixed_length && !col_metadata_[i].is_null_type) { + if (col_metadata_[i].fixed_length == 0) { + ARROW_ASSIGN_OR_RAISE(fixedlen_bufs[i], AllocatePaddedBitmap(num_groups)); + } else { + ARROW_ASSIGN_OR_RAISE( + fixedlen_bufs[i], + AllocatePaddedBuffer(num_groups * col_metadata_[i].fixed_length)); + } + } else { + ARROW_ASSIGN_OR_RAISE(fixedlen_bufs[i], + AllocatePaddedBuffer((num_groups + 1) * sizeof(uint32_t))); + } + cols_[i] = + KeyColumnArray(col_metadata_[i], num_groups, non_null_bufs[i]->mutable_data(), + fixedlen_bufs[i]->mutable_data(), nullptr); + } + + for (int64_t start_row = 0; start_row < num_groups;) { + int64_t batch_size_next = + std::min(num_groups - start_row, static_cast(minibatch_size_max_)); + encoder_.DecodeFixedLengthBuffers(start_row, start_row, batch_size_next, rows_, + &cols_); + start_row += batch_size_next; + } + + if (!rows_.metadata().is_fixed_length) { + for (size_t i = 0; i < num_columns; ++i) { + if (!col_metadata_[i].is_fixed_length) { + auto varlen_size = + reinterpret_cast(fixedlen_bufs[i]->data())[num_groups]; + ARROW_ASSIGN_OR_RAISE(varlen_bufs[i], AllocatePaddedBuffer(varlen_size)); + cols_[i] = KeyColumnArray( + col_metadata_[i], num_groups, non_null_bufs[i]->mutable_data(), + fixedlen_bufs[i]->mutable_data(), varlen_bufs[i]->mutable_data()); + } + } + + for (int64_t start_row = 0; start_row < num_groups;) { + int64_t batch_size_next = + std::min(num_groups - start_row, static_cast(minibatch_size_max_)); + encoder_.DecodeVaryingLengthBuffers(start_row, start_row, batch_size_next, rows_, + &cols_); + start_row += batch_size_next; + } + } + + ExecBatch out({}, num_groups); + out.values.resize(num_columns); + for (size_t i = 0; i < num_columns; ++i) { + if (col_metadata_[i].is_null_type) { + out.values[i] = ArrayData::Make(null(), num_groups, {nullptr}, num_groups); + continue; + } + auto valid_count = arrow::internal::CountSetBits( + non_null_bufs[i]->data(), /*offset=*/0, static_cast(num_groups)); + int null_count = static_cast(num_groups) - static_cast(valid_count); + + if (col_metadata_[i].is_fixed_length) { + out.values[i] = ArrayData::Make( + key_types_[i], num_groups, + {std::move(non_null_bufs[i]), std::move(fixedlen_bufs[i])}, null_count); + } else { + out.values[i] = + ArrayData::Make(key_types_[i], num_groups, + {std::move(non_null_bufs[i]), std::move(fixedlen_bufs[i]), + std::move(varlen_bufs[i])}, + null_count); + } + } + + // Process dictionaries + for (size_t icol = 0; icol < num_columns; ++icol) { + if (key_types_[icol]->id() == Type::DICTIONARY) { + if (dictionaries_[icol]) { + out.values[icol].array()->dictionary = dictionaries_[icol]->data(); + } else { + ARROW_ASSIGN_OR_RAISE(auto dict, MakeArrayOfNull(key_types_[icol], 0)); + out.values[icol].array()->dictionary = dict->data(); + } + } + } + + return out; + } + + static constexpr int log_minibatch_max_ = 10; + static constexpr int minibatch_size_max_ = 1 << log_minibatch_max_; + static constexpr int minibatch_size_min_ = 128; + int minibatch_size_; + + ExecContext* ctx_; + arrow::util::TempVectorStack temp_stack_; + LightContext encode_ctx_; + + std::vector> key_types_; + std::vector col_metadata_; + std::vector cols_; + std::vector minibatch_hashes_; + + std::vector> dictionaries_; + + RowTableImpl rows_; + RowTableImpl rows_minibatch_; + RowTableEncoder encoder_; + SwissTable map_; +}; + +} // namespace + +Result> Grouper::Make(const std::vector& descrs, + ExecContext* ctx) { + if (GrouperFastImpl::CanUse(descrs)) { + return GrouperFastImpl::Make(descrs, ctx); + } + return GrouperImpl::Make(descrs, ctx); +} + +Result> Grouper::ApplyGroupings(const ListArray& groupings, + const Array& array, + ExecContext* ctx) { + ARROW_ASSIGN_OR_RAISE(Datum sorted, + compute::Take(array, groupings.data()->child_data[0], + TakeOptions::NoBoundsCheck(), ctx)); + + return std::make_shared(list(array.type()), groupings.length(), + groupings.value_offsets(), sorted.make_array()); +} + +Result> Grouper::MakeGroupings(const UInt32Array& ids, + uint32_t num_groups, + ExecContext* ctx) { + if (ids.null_count() != 0) { + return Status::Invalid("MakeGroupings with null ids"); + } + + ARROW_ASSIGN_OR_RAISE(auto offsets, AllocateBuffer(sizeof(int32_t) * (num_groups + 1), + ctx->memory_pool())); + auto raw_offsets = reinterpret_cast(offsets->mutable_data()); + + std::memset(raw_offsets, 0, offsets->size()); + for (int i = 0; i < ids.length(); ++i) { + DCHECK_LT(ids.Value(i), num_groups); + raw_offsets[ids.Value(i)] += 1; + } + int32_t length = 0; + for (uint32_t id = 0; id < num_groups; ++id) { + auto offset = raw_offsets[id]; + raw_offsets[id] = length; + length += offset; + } + raw_offsets[num_groups] = length; + DCHECK_EQ(ids.length(), length); + + ARROW_ASSIGN_OR_RAISE(auto offsets_copy, + offsets->CopySlice(0, offsets->size(), ctx->memory_pool())); + raw_offsets = reinterpret_cast(offsets_copy->mutable_data()); + + ARROW_ASSIGN_OR_RAISE(auto sort_indices, AllocateBuffer(sizeof(int32_t) * ids.length(), + ctx->memory_pool())); + auto raw_sort_indices = reinterpret_cast(sort_indices->mutable_data()); + for (int i = 0; i < ids.length(); ++i) { + raw_sort_indices[raw_offsets[ids.Value(i)]++] = i; + } + + return std::make_shared( + list(int32()), num_groups, std::move(offsets), + std::make_shared(ids.length(), std::move(sort_indices))); +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/row/grouper.h b/cpp/src/arrow/compute/row/grouper.h new file mode 100644 index 00000000000..9de672e5adf --- /dev/null +++ b/cpp/src/arrow/compute/row/grouper.h @@ -0,0 +1,110 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include +#include + +#include "arrow/compute/exec.h" +#include "arrow/compute/exec/options.h" +#include "arrow/compute/kernel.h" +#include "arrow/datum.h" +#include "arrow/result.h" +#include "arrow/util/visibility.h" + +namespace arrow { +namespace compute { + +/// Consumes batches of keys and yields batches of the group ids. +class ARROW_EXPORT Grouper { + public: + virtual ~Grouper() = default; + + /// Construct a Grouper which receives the specified key types + static Result> Make(const std::vector& descrs, + ExecContext* ctx = default_exec_context()); + + /// Consume a batch of keys, producing the corresponding group ids as an integer array. + /// Currently only uint32 indices will be produced, eventually the bit width will only + /// be as wide as necessary. + virtual Result Consume(const ExecBatch& batch) = 0; + + /// Get current unique keys. May be called multiple times. + virtual Result GetUniques() = 0; + + /// Get the current number of groups. + virtual uint32_t num_groups() const = 0; + + /// \brief Assemble lists of indices of identical elements. + /// + /// \param[in] ids An unsigned, all-valid integral array which will be + /// used as grouping criteria. + /// \param[in] num_groups An upper bound for the elements of ids + /// \param[in] ctx Execution context to use during the operation + /// \return A num_groups-long ListArray where the slot at i contains a + /// list of indices where i appears in ids. + /// + /// MakeGroupings([ + /// 2, + /// 2, + /// 5, + /// 5, + /// 2, + /// 3 + /// ], 8) == [ + /// [], + /// [], + /// [0, 1, 4], + /// [5], + /// [], + /// [2, 3], + /// [], + /// [] + /// ] + static Result> MakeGroupings( + const UInt32Array& ids, uint32_t num_groups, + ExecContext* ctx = default_exec_context()); + + /// \brief Produce a ListArray whose slots are selections of `array` which correspond to + /// the provided groupings. + /// + /// For example, + /// ApplyGroupings([ + /// [], + /// [], + /// [0, 1, 4], + /// [5], + /// [], + /// [2, 3], + /// [], + /// [] + /// ], [2, 2, 5, 5, 2, 3]) == [ + /// [], + /// [], + /// [2, 2, 2], + /// [3], + /// [], + /// [5, 5], + /// [], + /// [] + /// ] + static Result> ApplyGroupings( + const ListArray& groupings, const Array& array, + ExecContext* ctx = default_exec_context()); +}; + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/row/row_internal.cc b/cpp/src/arrow/compute/row/row_internal.cc new file mode 100644 index 00000000000..c4e9cc99af5 --- /dev/null +++ b/cpp/src/arrow/compute/row/row_internal.cc @@ -0,0 +1,409 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/row/row_internal.h" + +#include "arrow/compute/exec/util.h" + +namespace arrow { +namespace compute { + +uint32_t RowTableMetadata::num_varbinary_cols() const { + uint32_t result = 0; + for (auto column_metadata : column_metadatas) { + if (!column_metadata.is_fixed_length) { + ++result; + } + } + return result; +} + +bool RowTableMetadata::is_compatible(const RowTableMetadata& other) const { + if (other.num_cols() != num_cols()) { + return false; + } + if (row_alignment != other.row_alignment || + string_alignment != other.string_alignment) { + return false; + } + for (size_t i = 0; i < column_metadatas.size(); ++i) { + if (column_metadatas[i].is_fixed_length != + other.column_metadatas[i].is_fixed_length) { + return false; + } + if (column_metadatas[i].fixed_length != other.column_metadatas[i].fixed_length) { + return false; + } + } + return true; +} + +void RowTableMetadata::FromColumnMetadataVector( + const std::vector& cols, int in_row_alignment, + int in_string_alignment) { + column_metadatas.resize(cols.size()); + for (size_t i = 0; i < cols.size(); ++i) { + column_metadatas[i] = cols[i]; + } + + const auto num_cols = static_cast(cols.size()); + + // Sort columns. + // + // Columns are sorted based on the size in bytes of their fixed-length part. + // For the varying-length column, the fixed-length part is the 32-bit field storing + // cumulative length of varying-length fields. + // + // The rules are: + // + // a) Boolean column, marked with fixed-length 0, is considered to have fixed-length + // part of 1 byte. + // + // b) Columns with fixed-length part being power of 2 or multiple of row + // alignment precede other columns. They are sorted in decreasing order of the size of + // their fixed-length part. + // + // c) Fixed-length columns precede varying-length columns when + // both have the same size fixed-length part. + // + column_order.resize(num_cols); + for (uint32_t i = 0; i < num_cols; ++i) { + column_order[i] = i; + } + std::sort( + column_order.begin(), column_order.end(), [&cols](uint32_t left, uint32_t right) { + bool is_left_pow2 = + !cols[left].is_fixed_length || ARROW_POPCOUNT64(cols[left].fixed_length) <= 1; + bool is_right_pow2 = !cols[right].is_fixed_length || + ARROW_POPCOUNT64(cols[right].fixed_length) <= 1; + bool is_left_fixedlen = cols[left].is_fixed_length; + bool is_right_fixedlen = cols[right].is_fixed_length; + uint32_t width_left = + cols[left].is_fixed_length ? cols[left].fixed_length : sizeof(uint32_t); + uint32_t width_right = + cols[right].is_fixed_length ? cols[right].fixed_length : sizeof(uint32_t); + if (is_left_pow2 != is_right_pow2) { + return is_left_pow2; + } + if (!is_left_pow2) { + return left < right; + } + if (width_left != width_right) { + return width_left > width_right; + } + if (is_left_fixedlen != is_right_fixedlen) { + return is_left_fixedlen; + } + return left < right; + }); + + row_alignment = in_row_alignment; + string_alignment = in_string_alignment; + varbinary_end_array_offset = 0; + + column_offsets.resize(num_cols); + uint32_t num_varbinary_cols = 0; + uint32_t offset_within_row = 0; + for (uint32_t i = 0; i < num_cols; ++i) { + const KeyColumnMetadata& col = cols[column_order[i]]; + if (col.is_fixed_length && col.fixed_length != 0 && + ARROW_POPCOUNT64(col.fixed_length) != 1) { + offset_within_row += RowTableMetadata::padding_for_alignment(offset_within_row, + string_alignment, col); + } + column_offsets[i] = offset_within_row; + if (!col.is_fixed_length) { + if (num_varbinary_cols == 0) { + varbinary_end_array_offset = offset_within_row; + } + DCHECK(column_offsets[i] - varbinary_end_array_offset == + num_varbinary_cols * sizeof(uint32_t)); + ++num_varbinary_cols; + offset_within_row += sizeof(uint32_t); + } else { + // Boolean column is a bit-vector, which is indicated by + // setting fixed length in column metadata to zero. + // It will be stored as a byte in output row. + if (col.fixed_length == 0) { + offset_within_row += 1; + } else { + offset_within_row += col.fixed_length; + } + } + } + + is_fixed_length = (num_varbinary_cols == 0); + fixed_length = + offset_within_row + + RowTableMetadata::padding_for_alignment( + offset_within_row, num_varbinary_cols == 0 ? row_alignment : string_alignment); + + // We set the number of bytes per row storing null masks of individual key columns + // to be a power of two. This is not required. It could be also set to the minimal + // number of bytes required for a given number of bits (one bit per column). + null_masks_bytes_per_row = 1; + while (static_cast(null_masks_bytes_per_row * 8) < num_cols) { + null_masks_bytes_per_row *= 2; + } +} + +RowTableImpl::RowTableImpl() : pool_(nullptr), rows_capacity_(0), bytes_capacity_(0) {} + +Status RowTableImpl::Init(MemoryPool* pool, const RowTableMetadata& metadata) { + pool_ = pool; + metadata_ = metadata; + + DCHECK(!null_masks_ && !offsets_ && !rows_); + + constexpr int64_t kInitialRowsCapacity = 8; + constexpr int64_t kInitialBytesCapacity = 1024; + + // Null masks + ARROW_ASSIGN_OR_RAISE( + auto null_masks, + AllocateResizableBuffer(size_null_masks(kInitialRowsCapacity), pool_)); + null_masks_ = std::move(null_masks); + memset(null_masks_->mutable_data(), 0, size_null_masks(kInitialRowsCapacity)); + + // Offsets and rows + if (!metadata.is_fixed_length) { + ARROW_ASSIGN_OR_RAISE( + auto offsets, AllocateResizableBuffer(size_offsets(kInitialRowsCapacity), pool_)); + offsets_ = std::move(offsets); + memset(offsets_->mutable_data(), 0, size_offsets(kInitialRowsCapacity)); + reinterpret_cast(offsets_->mutable_data())[0] = 0; + + ARROW_ASSIGN_OR_RAISE( + auto rows, + AllocateResizableBuffer(size_rows_varying_length(kInitialBytesCapacity), pool_)); + rows_ = std::move(rows); + memset(rows_->mutable_data(), 0, size_rows_varying_length(kInitialBytesCapacity)); + bytes_capacity_ = + size_rows_varying_length(kInitialBytesCapacity) - padding_for_vectors; + } else { + ARROW_ASSIGN_OR_RAISE( + auto rows, + AllocateResizableBuffer(size_rows_fixed_length(kInitialRowsCapacity), pool_)); + rows_ = std::move(rows); + memset(rows_->mutable_data(), 0, size_rows_fixed_length(kInitialRowsCapacity)); + bytes_capacity_ = size_rows_fixed_length(kInitialRowsCapacity) - padding_for_vectors; + } + + update_buffer_pointers(); + + rows_capacity_ = kInitialRowsCapacity; + + num_rows_ = 0; + num_rows_for_has_any_nulls_ = 0; + has_any_nulls_ = false; + + return Status::OK(); +} + +void RowTableImpl::Clean() { + num_rows_ = 0; + num_rows_for_has_any_nulls_ = 0; + has_any_nulls_ = false; + + if (!metadata_.is_fixed_length) { + reinterpret_cast(offsets_->mutable_data())[0] = 0; + } +} + +int64_t RowTableImpl::size_null_masks(int64_t num_rows) { + return num_rows * metadata_.null_masks_bytes_per_row + padding_for_vectors; +} + +int64_t RowTableImpl::size_offsets(int64_t num_rows) { + return (num_rows + 1) * sizeof(uint32_t) + padding_for_vectors; +} + +int64_t RowTableImpl::size_rows_fixed_length(int64_t num_rows) { + return num_rows * metadata_.fixed_length + padding_for_vectors; +} + +int64_t RowTableImpl::size_rows_varying_length(int64_t num_bytes) { + return num_bytes + padding_for_vectors; +} + +void RowTableImpl::update_buffer_pointers() { + buffers_[0] = mutable_buffers_[0] = null_masks_->mutable_data(); + if (metadata_.is_fixed_length) { + buffers_[1] = mutable_buffers_[1] = rows_->mutable_data(); + buffers_[2] = mutable_buffers_[2] = nullptr; + } else { + buffers_[1] = mutable_buffers_[1] = offsets_->mutable_data(); + buffers_[2] = mutable_buffers_[2] = rows_->mutable_data(); + } +} + +Status RowTableImpl::ResizeFixedLengthBuffers(int64_t num_extra_rows) { + if (rows_capacity_ >= num_rows_ + num_extra_rows) { + return Status::OK(); + } + + int64_t rows_capacity_new = std::max(static_cast(1), 2 * rows_capacity_); + while (rows_capacity_new < num_rows_ + num_extra_rows) { + rows_capacity_new *= 2; + } + + // Null masks + RETURN_NOT_OK(null_masks_->Resize(size_null_masks(rows_capacity_new), false)); + memset(null_masks_->mutable_data() + size_null_masks(rows_capacity_), 0, + size_null_masks(rows_capacity_new) - size_null_masks(rows_capacity_)); + + // Either offsets or rows + if (!metadata_.is_fixed_length) { + RETURN_NOT_OK(offsets_->Resize(size_offsets(rows_capacity_new), false)); + memset(offsets_->mutable_data() + size_offsets(rows_capacity_), 0, + size_offsets(rows_capacity_new) - size_offsets(rows_capacity_)); + } else { + RETURN_NOT_OK(rows_->Resize(size_rows_fixed_length(rows_capacity_new), false)); + memset(rows_->mutable_data() + size_rows_fixed_length(rows_capacity_), 0, + size_rows_fixed_length(rows_capacity_new) - + size_rows_fixed_length(rows_capacity_)); + bytes_capacity_ = size_rows_fixed_length(rows_capacity_new) - padding_for_vectors; + } + + update_buffer_pointers(); + + rows_capacity_ = rows_capacity_new; + + return Status::OK(); +} + +Status RowTableImpl::ResizeOptionalVaryingLengthBuffer(int64_t num_extra_bytes) { + int64_t num_bytes = offsets()[num_rows_]; + if (bytes_capacity_ >= num_bytes + num_extra_bytes || metadata_.is_fixed_length) { + return Status::OK(); + } + + int64_t bytes_capacity_new = std::max(static_cast(1), 2 * bytes_capacity_); + while (bytes_capacity_new < num_bytes + num_extra_bytes) { + bytes_capacity_new *= 2; + } + + RETURN_NOT_OK(rows_->Resize(size_rows_varying_length(bytes_capacity_new), false)); + memset(rows_->mutable_data() + size_rows_varying_length(bytes_capacity_), 0, + size_rows_varying_length(bytes_capacity_new) - + size_rows_varying_length(bytes_capacity_)); + + update_buffer_pointers(); + + bytes_capacity_ = bytes_capacity_new; + + return Status::OK(); +} + +Status RowTableImpl::AppendSelectionFrom(const RowTableImpl& from, + uint32_t num_rows_to_append, + const uint16_t* source_row_ids) { + DCHECK(metadata_.is_compatible(from.metadata())); + + RETURN_NOT_OK(ResizeFixedLengthBuffers(num_rows_to_append)); + + if (!metadata_.is_fixed_length) { + // Varying-length rows + auto from_offsets = reinterpret_cast(from.offsets_->data()); + auto to_offsets = reinterpret_cast(offsets_->mutable_data()); + uint32_t total_length = to_offsets[num_rows_]; + uint32_t total_length_to_append = 0; + for (uint32_t i = 0; i < num_rows_to_append; ++i) { + uint16_t row_id = source_row_ids ? source_row_ids[i] : i; + uint32_t length = from_offsets[row_id + 1] - from_offsets[row_id]; + total_length_to_append += length; + to_offsets[num_rows_ + i + 1] = total_length + total_length_to_append; + } + + RETURN_NOT_OK(ResizeOptionalVaryingLengthBuffer(total_length_to_append)); + + const uint8_t* src = from.rows_->data(); + uint8_t* dst = rows_->mutable_data() + total_length; + for (uint32_t i = 0; i < num_rows_to_append; ++i) { + uint16_t row_id = source_row_ids ? source_row_ids[i] : i; + uint32_t length = from_offsets[row_id + 1] - from_offsets[row_id]; + auto src64 = reinterpret_cast(src + from_offsets[row_id]); + auto dst64 = reinterpret_cast(dst); + for (uint32_t j = 0; j < bit_util::CeilDiv(length, 8); ++j) { + dst64[j] = src64[j]; + } + dst += length; + } + } else { + // Fixed-length rows + const uint8_t* src = from.rows_->data(); + uint8_t* dst = rows_->mutable_data() + num_rows_ * metadata_.fixed_length; + for (uint32_t i = 0; i < num_rows_to_append; ++i) { + uint16_t row_id = source_row_ids ? source_row_ids[i] : i; + uint32_t length = metadata_.fixed_length; + auto src64 = reinterpret_cast(src + length * row_id); + auto dst64 = reinterpret_cast(dst); + for (uint32_t j = 0; j < bit_util::CeilDiv(length, 8); ++j) { + dst64[j] = src64[j]; + } + dst += length; + } + } + + // Null masks + uint32_t byte_length = metadata_.null_masks_bytes_per_row; + uint64_t dst_byte_offset = num_rows_ * byte_length; + const uint8_t* src_base = from.null_masks_->data(); + uint8_t* dst_base = null_masks_->mutable_data(); + for (uint32_t i = 0; i < num_rows_to_append; ++i) { + uint32_t row_id = source_row_ids ? source_row_ids[i] : i; + int64_t src_byte_offset = row_id * byte_length; + const uint8_t* src = src_base + src_byte_offset; + uint8_t* dst = dst_base + dst_byte_offset; + for (uint32_t ibyte = 0; ibyte < byte_length; ++ibyte) { + dst[ibyte] = src[ibyte]; + } + dst_byte_offset += byte_length; + } + + num_rows_ += num_rows_to_append; + + return Status::OK(); +} + +Status RowTableImpl::AppendEmpty(uint32_t num_rows_to_append, + uint32_t num_extra_bytes_to_append) { + RETURN_NOT_OK(ResizeFixedLengthBuffers(num_rows_to_append)); + RETURN_NOT_OK(ResizeOptionalVaryingLengthBuffer(num_extra_bytes_to_append)); + num_rows_ += num_rows_to_append; + if (metadata_.row_alignment > 1 || metadata_.string_alignment > 1) { + memset(rows_->mutable_data(), 0, bytes_capacity_); + } + return Status::OK(); +} + +bool RowTableImpl::has_any_nulls(const LightContext* ctx) const { + if (has_any_nulls_) { + return true; + } + if (num_rows_for_has_any_nulls_ < num_rows_) { + auto size_per_row = metadata().null_masks_bytes_per_row; + has_any_nulls_ = !util::bit_util::are_all_bytes_zero( + ctx->hardware_flags, null_masks() + size_per_row * num_rows_for_has_any_nulls_, + static_cast(size_per_row * (num_rows_ - num_rows_for_has_any_nulls_))); + num_rows_for_has_any_nulls_ = num_rows_; + } + return has_any_nulls_; +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/row/row_internal.h b/cpp/src/arrow/compute/row/row_internal.h new file mode 100644 index 00000000000..6b53c72b45c --- /dev/null +++ b/cpp/src/arrow/compute/row/row_internal.h @@ -0,0 +1,244 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +#pragma once + +#include +#include + +#include "arrow/buffer.h" +#include "arrow/compute/light_array.h" +#include "arrow/memory_pool.h" +#include "arrow/status.h" +#include "arrow/util/logging.h" + +namespace arrow { +namespace compute { + +/// Description of the data stored in a RowTable +struct ARROW_EXPORT RowTableMetadata { + /// \brief True if there are no variable length columns in the table + bool is_fixed_length; + + /// For a fixed-length binary row, common size of rows in bytes, + /// rounded up to the multiple of alignment. + /// + /// For a varying-length binary, size of all encoded fixed-length key columns, + /// including lengths of varying-length columns, rounded up to the multiple of string + /// alignment. + uint32_t fixed_length; + + /// Offset within a row to the array of 32-bit offsets within a row of + /// ends of varbinary fields. + /// Used only when the row is not fixed-length, zero for fixed-length row. + /// There are N elements for N varbinary fields. + /// Each element is the offset within a row of the first byte after + /// the corresponding varbinary field bytes in that row. + /// If varbinary fields begin at aligned addresses, than the end of the previous + /// varbinary field needs to be rounded up according to the specified alignment + /// to obtain the beginning of the next varbinary field. + /// The first varbinary field starts at offset specified by fixed_length, + /// which should already be aligned. + uint32_t varbinary_end_array_offset; + + /// Fixed number of bytes per row that are used to encode null masks. + /// Null masks indicate for a single row which of its columns are null. + /// Nth bit in the sequence of bytes assigned to a row represents null + /// information for Nth field according to the order in which they are encoded. + int null_masks_bytes_per_row; + + /// Power of 2. Every row will start at an offset aligned to that number of bytes. + int row_alignment; + + /// Power of 2. Must be no greater than row alignment. + /// Every non-power-of-2 binary field and every varbinary field bytes + /// will start aligned to that number of bytes. + int string_alignment; + + /// Metadata of encoded columns in their original order. + std::vector column_metadatas; + + /// Order in which fields are encoded. + std::vector column_order; + + /// Offsets within a row to fields in their encoding order. + std::vector column_offsets; + + /// Rounding up offset to the nearest multiple of alignment value. + /// Alignment must be a power of 2. + static inline uint32_t padding_for_alignment(uint32_t offset, int required_alignment) { + ARROW_DCHECK(ARROW_POPCOUNT64(required_alignment) == 1); + return static_cast((-static_cast(offset)) & + (required_alignment - 1)); + } + + /// Rounding up offset to the beginning of next column, + /// choosing required alignment based on the data type of that column. + static inline uint32_t padding_for_alignment(uint32_t offset, int string_alignment, + const KeyColumnMetadata& col_metadata) { + if (!col_metadata.is_fixed_length || + ARROW_POPCOUNT64(col_metadata.fixed_length) <= 1) { + return 0; + } else { + return padding_for_alignment(offset, string_alignment); + } + } + + /// Returns an array of offsets within a row of ends of varbinary fields. + inline const uint32_t* varbinary_end_array(const uint8_t* row) const { + ARROW_DCHECK(!is_fixed_length); + return reinterpret_cast(row + varbinary_end_array_offset); + } + + /// \brief An array of mutable offsets within a row of ends of varbinary fields. + inline uint32_t* varbinary_end_array(uint8_t* row) const { + ARROW_DCHECK(!is_fixed_length); + return reinterpret_cast(row + varbinary_end_array_offset); + } + + /// Returns the offset within the row and length of the first varbinary field. + inline void first_varbinary_offset_and_length(const uint8_t* row, uint32_t* offset, + uint32_t* length) const { + ARROW_DCHECK(!is_fixed_length); + *offset = fixed_length; + *length = varbinary_end_array(row)[0] - fixed_length; + } + + /// Returns the offset within the row and length of the second and further varbinary + /// fields. + inline void nth_varbinary_offset_and_length(const uint8_t* row, int varbinary_id, + uint32_t* out_offset, + uint32_t* out_length) const { + ARROW_DCHECK(!is_fixed_length); + ARROW_DCHECK(varbinary_id > 0); + const uint32_t* varbinary_end = varbinary_end_array(row); + uint32_t offset = varbinary_end[varbinary_id - 1]; + offset += padding_for_alignment(offset, string_alignment); + *out_offset = offset; + *out_length = varbinary_end[varbinary_id] - offset; + } + + uint32_t encoded_field_order(uint32_t icol) const { return column_order[icol]; } + + uint32_t encoded_field_offset(uint32_t icol) const { return column_offsets[icol]; } + + uint32_t num_cols() const { return static_cast(column_metadatas.size()); } + + uint32_t num_varbinary_cols() const; + + /// \brief Populate this instance to describe `cols` with the given alignment + void FromColumnMetadataVector(const std::vector& cols, + int in_row_alignment, int in_string_alignment); + + /// \brief True if `other` has the same number of columns + /// and each column has the same width (two variable length + /// columns are considered to have the same width) + bool is_compatible(const RowTableMetadata& other) const; +}; + +/// \brief A table of data stored in row-major order +/// +/// Can only store non-nested data types +/// +/// Can store both fixed-size data types and variable-length data types +class ARROW_EXPORT RowTableImpl { + public: + RowTableImpl(); + /// \brief Initialize a row array for use + /// + /// This must be called before any other method + Status Init(MemoryPool* pool, const RowTableMetadata& metadata); + /// \brief Clear all rows from the table + /// + /// Does not shrink buffers + void Clean(); + /// \brief Add empty rows + /// \param num_rows_to_append The number of empty rows to append + /// \param num_extra_bytes_to_append For tables storing variable-length data this + /// should be a guess of how many data bytes will be needed to populate the + /// data. This is ignored if there are no variable-length columns + Status AppendEmpty(uint32_t num_rows_to_append, uint32_t num_extra_bytes_to_append); + /// \brief Append rows from a source table + /// \param from The table to append from + /// \param num_rows_to_append The number of rows to append + /// \param source_row_ids Indices (into `from`) of the desired rows + Status AppendSelectionFrom(const RowTableImpl& from, uint32_t num_rows_to_append, + const uint16_t* source_row_ids); + /// \brief Metadata describing the data stored in this table + const RowTableMetadata& metadata() const { return metadata_; } + /// \brief The number of rows stored in the table + int64_t length() const { return num_rows_; } + // Accessors into the table's buffers + const uint8_t* data(int i) const { + ARROW_DCHECK(i >= 0 && i <= max_buffers_); + return buffers_[i]; + } + uint8_t* mutable_data(int i) { + ARROW_DCHECK(i >= 0 && i <= max_buffers_); + return mutable_buffers_[i]; + } + const uint32_t* offsets() const { return reinterpret_cast(data(1)); } + uint32_t* mutable_offsets() { return reinterpret_cast(mutable_data(1)); } + const uint8_t* null_masks() const { return null_masks_->data(); } + uint8_t* null_masks() { return null_masks_->mutable_data(); } + + /// \brief True if there is a null value anywhere in the table + /// + /// This calculation is memoized based on the number of rows and assumes + /// that values are only appended (and not modified in place) between + /// successive calls + bool has_any_nulls(const LightContext* ctx) const; + + private: + Status ResizeFixedLengthBuffers(int64_t num_extra_rows); + Status ResizeOptionalVaryingLengthBuffer(int64_t num_extra_bytes); + + // Helper functions to determine the number of bytes needed for each + // buffer given a number of rows. + int64_t size_null_masks(int64_t num_rows); + int64_t size_offsets(int64_t num_rows); + int64_t size_rows_fixed_length(int64_t num_rows); + int64_t size_rows_varying_length(int64_t num_bytes); + // Called after resize to fix pointers + void update_buffer_pointers(); + + static constexpr int64_t padding_for_vectors = 64; + MemoryPool* pool_; + RowTableMetadata metadata_; + // Buffers can only expand during lifetime and never shrink. + std::unique_ptr null_masks_; + // Only used if the table has variable-length columns + // Stores the offsets into the binary data + std::unique_ptr offsets_; + // Stores the fixed-length parts of the rows + std::unique_ptr rows_; + static constexpr int max_buffers_ = 3; + const uint8_t* buffers_[max_buffers_]; + uint8_t* mutable_buffers_[max_buffers_]; + // The number of rows in the table + int64_t num_rows_; + // The number of rows that can be stored in the table without resizing + int64_t rows_capacity_; + // The number of bytes that can be stored in the table without resizing + int64_t bytes_capacity_; + + // Mutable to allow lazy evaluation + mutable int64_t num_rows_for_has_any_nulls_; + mutable bool has_any_nulls_; +}; + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/dataset/partition.cc b/cpp/src/arrow/dataset/partition.cc index 763ea0bc6ce..bedafa28c8c 100644 --- a/cpp/src/arrow/dataset/partition.cc +++ b/cpp/src/arrow/dataset/partition.cc @@ -26,11 +26,11 @@ #include "arrow/array/array_dict.h" #include "arrow/array/array_nested.h" #include "arrow/array/builder_dict.h" -#include "arrow/compute/api_aggregate.h" #include "arrow/compute/api_scalar.h" #include "arrow/compute/api_vector.h" #include "arrow/compute/cast.h" #include "arrow/compute/exec/expression_internal.h" +#include "arrow/compute/row/grouper.h" #include "arrow/dataset/dataset_internal.h" #include "arrow/filesystem/path_util.h" #include "arrow/scalar.h" @@ -141,14 +141,13 @@ Result KeyValuePartitioning::Partition( key_batch.values.emplace_back(batch->column_data(i)); } - ARROW_ASSIGN_OR_RAISE(auto grouper, - compute::internal::Grouper::Make(key_batch.GetDescriptors())); + ARROW_ASSIGN_OR_RAISE(auto grouper, compute::Grouper::Make(key_batch.GetDescriptors())); ARROW_ASSIGN_OR_RAISE(Datum id_batch, grouper->Consume(key_batch)); auto ids = id_batch.array_as(); - ARROW_ASSIGN_OR_RAISE(auto groupings, compute::internal::Grouper::MakeGroupings( - *ids, grouper->num_groups())); + ARROW_ASSIGN_OR_RAISE(auto groupings, + compute::Grouper::MakeGroupings(*ids, grouper->num_groups())); ARROW_ASSIGN_OR_RAISE(auto uniques, grouper->GetUniques()); ArrayVector unique_arrays(num_keys); From 98d6fbeee3e1041297f89f1816a2e678faf33e97 Mon Sep 17 00:00:00 2001 From: michalursa Date: Mon, 23 May 2022 12:25:33 -1000 Subject: [PATCH 2/3] ARROW-16590: Add new capabilities to key encoder and key compare. --- cpp/src/arrow/CMakeLists.txt | 1 + cpp/src/arrow/compute/exec/util.h | 49 +- cpp/src/arrow/compute/row/compare_internal.cc | 120 +++-- cpp/src/arrow/compute/row/compare_internal.h | 25 +- .../compute/row/compare_internal_avx2.cc | 95 +++- cpp/src/arrow/compute/row/encode.cc | 456 ++++++++++++++++++ cpp/src/arrow/compute/row/encode.h | 142 ++++++ cpp/src/arrow/compute/row/encode_internal.cc | 102 +++- cpp/src/arrow/compute/row/encode_internal.h | 156 +++++- .../arrow/compute/row/encode_internal_avx2.cc | 166 +++++++ cpp/src/arrow/compute/row/grouper.cc | 10 +- cpp/src/arrow/compute/row/row_internal.cc | 5 + cpp/src/arrow/compute/row/row_internal.h | 3 + 13 files changed, 1225 insertions(+), 105 deletions(-) create mode 100644 cpp/src/arrow/compute/row/encode.cc create mode 100644 cpp/src/arrow/compute/row/encode.h diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index d8e662b63f1..96a3806f368 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -440,6 +440,7 @@ if(ARROW_COMPUTE) compute/kernels/vector_replace.cc compute/kernels/vector_selection.cc compute/kernels/vector_sort.cc + compute/row/encode.cc compute/row/encode_internal.cc compute/row/compare_internal.cc compute/row/grouper.cc diff --git a/cpp/src/arrow/compute/exec/util.h b/cpp/src/arrow/compute/exec/util.h index 839a8a7d29c..1a635857f91 100644 --- a/cpp/src/arrow/compute/exec/util.h +++ b/cpp/src/arrow/compute/exec/util.h @@ -77,7 +77,8 @@ using int64_for_gather_t = const long long int; // NOLINT runtime-int // class MiniBatch { public: - static constexpr int kMiniBatchLength = 1024; + static constexpr int kLogMiniBatchLength = 10; + static constexpr int kMiniBatchLength = 1 << kLogMiniBatchLength; }; /// Storage used to allocate temporary vectors of a batch size. @@ -295,5 +296,51 @@ class ARROW_EXPORT ThreadIndexer { std::unordered_map id_to_index_; }; +// Helper class to calculate the modified number of rows to process using SIMD. +// +// Some array elements at the end will be skipped in order to avoid buffer +// overrun, when doing memory loads and stores using larger word size than a +// single array element. +// +class TailSkipForSIMD { + public: + static int64_t FixBitAccess(int num_bytes_accessed_together, int64_t num_rows, + int bit_offset) { + int64_t num_bytes = bit_util::BytesForBits(num_rows + bit_offset); + int64_t num_bytes_safe = + std::max(static_cast(0LL), num_bytes - num_bytes_accessed_together + 1); + int64_t num_rows_safe = + std::max(static_cast(0LL), 8 * num_bytes_safe - bit_offset); + return std::min(num_rows_safe, num_rows); + } + static int64_t FixBinaryAccess(int num_bytes_accessed_together, int64_t num_rows, + int64_t length) { + int64_t num_rows_to_skip = bit_util::CeilDiv(length, num_bytes_accessed_together); + int64_t num_rows_safe = + std::max(static_cast(0LL), num_rows - num_rows_to_skip); + return num_rows_safe; + } + static int64_t FixVarBinaryAccess(int num_bytes_accessed_together, int64_t num_rows, + const uint32_t* offsets) { + // Do not process rows that could read past the end of the buffer using N + // byte loads/stores. + // + int64_t num_rows_safe = num_rows; + while (num_rows_safe > 0 && + offsets[num_rows_safe] + num_bytes_accessed_together > offsets[num_rows]) { + --num_rows_safe; + } + return num_rows_safe; + } + static int FixSelection(int64_t num_rows_safe, int num_selected, + const uint16_t* selection) { + int num_selected_safe = num_selected; + while (num_selected_safe > 0 && selection[num_selected_safe] >= num_rows_safe) { + --num_selected_safe; + } + return num_selected_safe; + } +}; + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/row/compare_internal.cc b/cpp/src/arrow/compute/row/compare_internal.cc index e863c9cd05f..773a8126f96 100644 --- a/cpp/src/arrow/compute/row/compare_internal.cc +++ b/cpp/src/arrow/compute/row/compare_internal.cc @@ -35,7 +35,8 @@ void KeyCompare::NullUpdateColumnToRow(uint32_t id_col, uint32_t num_rows_to_com const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, - uint8_t* match_bytevector) { + uint8_t* match_bytevector, + bool are_cols_in_encoding_order) { if (!rows.has_any_nulls(ctx) && !col.data(0)) { return; } @@ -48,6 +49,9 @@ void KeyCompare::NullUpdateColumnToRow(uint32_t id_col, uint32_t num_rows_to_com } #endif + uint32_t null_bit_id = + are_cols_in_encoding_order ? id_col : rows.metadata().pos_after_encoding(id_col); + if (!col.data(0)) { // Remove rows from the result for which the column value is a null const uint8_t* null_masks = rows.null_masks(); @@ -55,11 +59,12 @@ void KeyCompare::NullUpdateColumnToRow(uint32_t id_col, uint32_t num_rows_to_com for (uint32_t i = num_processed; i < num_rows_to_compare; ++i) { uint32_t irow_left = use_selection ? sel_left_maybe_null[i] : i; uint32_t irow_right = left_to_right_map[irow_left]; - int64_t bitid = irow_right * null_mask_num_bytes * 8 + id_col; + int64_t bitid = irow_right * null_mask_num_bytes * 8 + null_bit_id; match_bytevector[i] &= (bit_util::GetBit(null_masks, bitid) ? 0 : 0xff); } } else if (!rows.has_any_nulls(ctx)) { - // Remove rows from the result for which the column value on left side is null + // Remove rows from the result for which the column value on left side is + // null const uint8_t* non_nulls = col.data(0); ARROW_DCHECK(non_nulls); for (uint32_t i = num_processed; i < num_rows_to_compare; ++i) { @@ -75,7 +80,7 @@ void KeyCompare::NullUpdateColumnToRow(uint32_t id_col, uint32_t num_rows_to_com for (uint32_t i = num_processed; i < num_rows_to_compare; ++i) { uint32_t irow_left = use_selection ? sel_left_maybe_null[i] : i; uint32_t irow_right = left_to_right_map[irow_left]; - int64_t bitid_right = irow_right * null_mask_num_bytes * 8 + id_col; + int64_t bitid_right = irow_right * null_mask_num_bytes * 8 + null_bit_id; int right_null = bit_util::GetBit(null_masks, bitid_right) ? 0xff : 0; int left_null = bit_util::GetBit(non_nulls, irow_left + col.bit_offset(0)) ? 0 : 0xff; @@ -228,27 +233,16 @@ void KeyCompare::CompareBinaryColumnToRow(uint32_t offset_within_row, // Overwrites the match_bytevector instead of updating it template -void KeyCompare::CompareVarBinaryColumnToRow(uint32_t id_varbinary_col, - uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, - LightContext* ctx, const KeyColumnArray& col, - const RowTableImpl& rows, - uint8_t* match_bytevector) { -#if defined(ARROW_HAVE_AVX2) - if (ctx->has_avx2()) { - CompareVarBinaryColumnToRow_avx2( - use_selection, is_first_varbinary_col, id_varbinary_col, num_rows_to_compare, - sel_left_maybe_null, left_to_right_map, ctx, col, rows, match_bytevector); - return; - } -#endif - +void KeyCompare::CompareVarBinaryColumnToRowHelper( + uint32_t id_varbinary_col, uint32_t first_row_to_compare, + uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, uint8_t* match_bytevector) { const uint32_t* offsets_left = col.offsets(); const uint32_t* offsets_right = rows.offsets(); const uint8_t* rows_left = col.data(2); const uint8_t* rows_right = rows.data(2); - for (uint32_t i = 0; i < num_rows_to_compare; ++i) { + for (uint32_t i = first_row_to_compare; i < num_rows_to_compare; ++i) { uint32_t irow_left = use_selection ? sel_left_maybe_null[i] : i; uint32_t irow_right = left_to_right_map[irow_left]; uint32_t begin_left = offsets_left[irow_left]; @@ -292,6 +286,29 @@ void KeyCompare::CompareVarBinaryColumnToRow(uint32_t id_varbinary_col, } } +// Overwrites the match_bytevector instead of updating it +template +void KeyCompare::CompareVarBinaryColumnToRow(uint32_t id_varbinary_col, + uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, + LightContext* ctx, const KeyColumnArray& col, + const RowTableImpl& rows, + uint8_t* match_bytevector) { + uint32_t num_processed = 0; +#if defined(ARROW_HAVE_AVX2) + if (ctx->has_avx2()) { + num_processed = CompareVarBinaryColumnToRow_avx2( + use_selection, is_first_varbinary_col, id_varbinary_col, num_rows_to_compare, + sel_left_maybe_null, left_to_right_map, ctx, col, rows, match_bytevector); + } +#endif + + CompareVarBinaryColumnToRowHelper( + id_varbinary_col, num_processed, num_rows_to_compare, sel_left_maybe_null, + left_to_right_map, ctx, col, rows, match_bytevector); +} + void KeyCompare::AndByteVectors(LightContext* ctx, uint32_t num_elements, uint8_t* bytevector_A, const uint8_t* bytevector_B) { uint32_t num_processed = 0; @@ -308,13 +325,12 @@ void KeyCompare::AndByteVectors(LightContext* ctx, uint32_t num_elements, } } -void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, - LightContext* ctx, uint32_t* out_num_rows, - uint16_t* out_sel_left_maybe_same, - const std::vector& cols, - const RowTableImpl& rows) { +void KeyCompare::CompareColumnsToRows( + uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, LightContext* ctx, uint32_t* out_num_rows, + uint16_t* out_sel_left_maybe_same, const std::vector& cols, + const RowTableImpl& rows, bool are_cols_in_encoding_order, + uint8_t* out_match_bitvector_maybe_null) { if (num_rows_to_compare == 0) { *out_num_rows = 0; return; @@ -343,8 +359,11 @@ void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, } continue; } - uint32_t offset_within_row = - rows.metadata().encoded_field_offset(static_cast(icol)); + + uint32_t offset_within_row = rows.metadata().encoded_field_offset( + are_cols_in_encoding_order + ? static_cast(icol) + : rows.metadata().pos_after_encoding(static_cast(icol))); if (col.metadata().is_fixed_length) { if (sel_left_maybe_null) { CompareBinaryColumnToRow( @@ -354,7 +373,8 @@ void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, NullUpdateColumnToRow( static_cast(icol), num_rows_to_compare, sel_left_maybe_null, left_to_right_map, ctx, col, rows, - is_first_column ? match_bytevector_A : match_bytevector_B); + is_first_column ? match_bytevector_A : match_bytevector_B, + are_cols_in_encoding_order); } else { // Version without using selection vector CompareBinaryColumnToRow( @@ -364,7 +384,8 @@ void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, NullUpdateColumnToRow( static_cast(icol), num_rows_to_compare, sel_left_maybe_null, left_to_right_map, ctx, col, rows, - is_first_column ? match_bytevector_A : match_bytevector_B); + is_first_column ? match_bytevector_A : match_bytevector_B, + are_cols_in_encoding_order); } if (!is_first_column) { AndByteVectors(ctx, num_rows_to_compare, match_bytevector_A, match_bytevector_B); @@ -391,7 +412,8 @@ void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, NullUpdateColumnToRow( static_cast(icol), num_rows_to_compare, sel_left_maybe_null, left_to_right_map, ctx, col, rows, - is_first_column ? match_bytevector_A : match_bytevector_B); + is_first_column ? match_bytevector_A : match_bytevector_B, + are_cols_in_encoding_order); } else { if (ivarbinary == 0) { CompareVarBinaryColumnToRow( @@ -405,7 +427,8 @@ void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, NullUpdateColumnToRow( static_cast(icol), num_rows_to_compare, sel_left_maybe_null, left_to_right_map, ctx, col, rows, - is_first_column ? match_bytevector_A : match_bytevector_B); + is_first_column ? match_bytevector_A : match_bytevector_B, + are_cols_in_encoding_order); } if (!is_first_column) { AndByteVectors(ctx, num_rows_to_compare, match_bytevector_A, match_bytevector_B); @@ -417,18 +440,25 @@ void KeyCompare::CompareColumnsToRows(uint32_t num_rows_to_compare, util::bit_util::bytes_to_bits(ctx->hardware_flags, num_rows_to_compare, match_bytevector_A, match_bitvector); - if (sel_left_maybe_null) { - int out_num_rows_int; - util::bit_util::bits_filter_indexes(0, ctx->hardware_flags, num_rows_to_compare, - match_bitvector, sel_left_maybe_null, - &out_num_rows_int, out_sel_left_maybe_same); - *out_num_rows = out_num_rows_int; + if (out_match_bitvector_maybe_null) { + ARROW_DCHECK(out_num_rows == nullptr); + ARROW_DCHECK(out_sel_left_maybe_same == nullptr); + memcpy(out_match_bitvector_maybe_null, match_bitvector, + bit_util::BytesForBits(num_rows_to_compare)); } else { - int out_num_rows_int; - util::bit_util::bits_to_indexes(0, ctx->hardware_flags, num_rows_to_compare, - match_bitvector, &out_num_rows_int, - out_sel_left_maybe_same); - *out_num_rows = out_num_rows_int; + if (sel_left_maybe_null) { + int out_num_rows_int; + util::bit_util::bits_filter_indexes(0, ctx->hardware_flags, num_rows_to_compare, + match_bitvector, sel_left_maybe_null, + &out_num_rows_int, out_sel_left_maybe_same); + *out_num_rows = out_num_rows_int; + } else { + int out_num_rows_int; + util::bit_util::bits_to_indexes(0, ctx->hardware_flags, num_rows_to_compare, + match_bitvector, &out_num_rows_int, + out_sel_left_maybe_same); + *out_num_rows = out_num_rows_int; + } } } diff --git a/cpp/src/arrow/compute/row/compare_internal.h b/cpp/src/arrow/compute/row/compare_internal.h index e3b9057115e..f9ec1e7f535 100644 --- a/cpp/src/arrow/compute/row/compare_internal.h +++ b/cpp/src/arrow/compute/row/compare_internal.h @@ -35,13 +35,12 @@ class KeyCompare { // Returns a single 16-bit selection vector of rows that failed comparison. // If there is input selection on the left, the resulting selection is a filtered image // of input selection. - static void CompareColumnsToRows(uint32_t num_rows_to_compare, - const uint16_t* sel_left_maybe_null, - const uint32_t* left_to_right_map, LightContext* ctx, - uint32_t* out_num_rows, - uint16_t* out_sel_left_maybe_same, - const std::vector& cols, - const RowTableImpl& rows); + static void CompareColumnsToRows( + uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, + const uint32_t* left_to_right_map, LightContext* ctx, uint32_t* out_num_rows, + uint16_t* out_sel_left_maybe_same, const std::vector& cols, + const RowTableImpl& rows, bool are_cols_in_encoding_order, + uint8_t* out_match_bitvector_maybe_null = NULLPTR); private: template @@ -49,7 +48,8 @@ class KeyCompare { const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, - uint8_t* match_bytevector); + uint8_t* match_bytevector, + bool are_cols_in_encoding_order); template static void CompareBinaryColumnToRowHelper( @@ -67,6 +67,13 @@ class KeyCompare { const RowTableImpl& rows, uint8_t* match_bytevector); + template + static void CompareVarBinaryColumnToRowHelper( + uint32_t id_varlen_col, uint32_t first_row_to_compare, uint32_t num_rows_to_compare, + const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, + LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, + uint8_t* match_bytevector); + template static void CompareVarBinaryColumnToRow(uint32_t id_varlen_col, uint32_t num_rows_to_compare, @@ -125,7 +132,7 @@ class KeyCompare { LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector); - static void CompareVarBinaryColumnToRow_avx2( + static uint32_t CompareVarBinaryColumnToRow_avx2( bool use_selection, bool is_first_varbinary_col, uint32_t id_varlen_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, diff --git a/cpp/src/arrow/compute/row/compare_internal_avx2.cc b/cpp/src/arrow/compute/row/compare_internal_avx2.cc index 818f4c4fe7f..96dacab6797 100644 --- a/cpp/src/arrow/compute/row/compare_internal_avx2.cc +++ b/cpp/src/arrow/compute/row/compare_internal_avx2.cc @@ -44,6 +44,9 @@ uint32_t KeyCompare::NullUpdateColumnToRowImp_avx2( if (!rows.has_any_nulls(ctx) && !col.data(0)) { return num_rows_to_compare; } + + uint32_t null_bit_id = rows.metadata().pos_after_encoding(id_col); + if (!col.data(0)) { // Remove rows from the result for which the column value is a null const uint8_t* null_masks = rows.null_masks(); @@ -63,7 +66,7 @@ uint32_t KeyCompare::NullUpdateColumnToRowImp_avx2( } __m256i bitid = _mm256_mullo_epi32(irow_right, _mm256_set1_epi32(null_mask_num_bytes * 8)); - bitid = _mm256_add_epi32(bitid, _mm256_set1_epi32(id_col)); + bitid = _mm256_add_epi32(bitid, _mm256_set1_epi32(null_bit_id)); __m256i right = _mm256_i32gather_epi32((const int*)null_masks, _mm256_srli_epi32(bitid, 3), 1); right = _mm256_and_si256( @@ -80,7 +83,8 @@ uint32_t KeyCompare::NullUpdateColumnToRowImp_avx2( num_processed = num_rows_to_compare / unroll * unroll; return num_processed; } else if (!rows.has_any_nulls(ctx)) { - // Remove rows from the result for which the column value on left side is null + // Remove rows from the result for which the column value on left side is + // null const uint8_t* non_nulls = col.data(0); ARROW_DCHECK(non_nulls); uint32_t num_processed = 0; @@ -145,7 +149,7 @@ uint32_t KeyCompare::NullUpdateColumnToRowImp_avx2( } __m256i bitid = _mm256_mullo_epi32(irow_right, _mm256_set1_epi32(null_mask_num_bytes * 8)); - bitid = _mm256_add_epi32(bitid, _mm256_set1_epi32(id_col)); + bitid = _mm256_add_epi32(bitid, _mm256_set1_epi32(null_bit_id)); __m256i right = _mm256_i32gather_epi32((const int*)null_masks, _mm256_srli_epi32(bitid, 3), 1); right = _mm256_and_si256( @@ -252,22 +256,22 @@ inline uint64_t CompareSelected8_avx2(const uint8_t* left_base, const uint8_t* r int bit_offset = 0) { __m256i left; switch (column_width) { - case 0: + case 0: { irow_left = _mm256_add_epi32(irow_left, _mm256_set1_epi32(bit_offset)); left = _mm256_i32gather_epi32((const int*)left_base, - _mm256_srli_epi32(irow_left, 3), 1); - left = _mm256_and_si256( - _mm256_set1_epi32(1), - _mm256_srlv_epi32(left, _mm256_and_si256(irow_left, _mm256_set1_epi32(7)))); - left = _mm256_mullo_epi32(left, _mm256_set1_epi32(0xff)); - break; + _mm256_srli_epi32(irow_left, 5), 4); + __m256i bit_selection = _mm256_sllv_epi32( + _mm256_set1_epi32(1), _mm256_and_si256(irow_left, _mm256_set1_epi32(31))); + left = _mm256_cmpeq_epi32(bit_selection, _mm256_and_si256(left, bit_selection)); + left = _mm256_and_si256(left, _mm256_set1_epi32(0xff)); + } break; case 1: left = _mm256_i32gather_epi32((const int*)left_base, irow_left, 1); left = _mm256_and_si256(left, _mm256_set1_epi32(0xff)); break; case 2: left = _mm256_i32gather_epi32((const int*)left_base, irow_left, 2); - left = _mm256_and_si256(left, _mm256_set1_epi32(0xff)); + left = _mm256_and_si256(left, _mm256_set1_epi32(0xffff)); break; case 4: left = _mm256_i32gather_epi32((const int*)left_base, irow_left, 4); @@ -311,15 +315,15 @@ inline uint64_t Compare8_avx2(const uint8_t* left_base, const uint8_t* right_bas } break; case 1: left = _mm256_cvtepu8_epi32(_mm_set1_epi64x( - reinterpret_cast(left_base)[irow_left_first / 8])); + *reinterpret_cast(left_base + irow_left_first))); break; case 2: left = _mm256_cvtepu16_epi32(_mm_loadu_si128( - reinterpret_cast(left_base) + irow_left_first / 8)); + reinterpret_cast(left_base + 2 * irow_left_first))); break; case 4: - left = _mm256_loadu_si256(reinterpret_cast(left_base) + - irow_left_first / 8); + left = _mm256_loadu_si256( + reinterpret_cast(left_base + 4 * irow_left_first)); break; default: ARROW_DCHECK(false); @@ -347,19 +351,17 @@ inline uint64_t Compare8_64bit_avx2(const uint8_t* left_base, const uint8_t* rig __m256i offset_right) { auto left_base_i64 = reinterpret_cast(left_base); - __m256i left_lo = - _mm256_i32gather_epi64(left_base_i64, _mm256_castsi256_si128(irow_left), 8); - __m256i left_hi = - _mm256_i32gather_epi64(left_base_i64, _mm256_extracti128_si256(irow_left, 1), 8); + __m256i left_lo, left_hi; if (use_selection) { left_lo = _mm256_i32gather_epi64(left_base_i64, _mm256_castsi256_si128(irow_left), 8); left_hi = _mm256_i32gather_epi64(left_base_i64, _mm256_extracti128_si256(irow_left, 1), 8); } else { - left_lo = _mm256_loadu_si256(reinterpret_cast(left_base) + - irow_left_first / 4); - left_hi = _mm256_loadu_si256(reinterpret_cast(left_base) + - irow_left_first / 4 + 1); + left_lo = _mm256_loadu_si256( + reinterpret_cast(left_base + irow_left_first * sizeof(uint64_t))); + left_hi = _mm256_loadu_si256( + reinterpret_cast(left_base + irow_left_first * sizeof(uint64_t)) + + 1); } auto right_base_i64 = reinterpret_cast(right_base); @@ -532,7 +534,7 @@ void KeyCompare::CompareVarBinaryColumnToRowImp_avx2( const __m256i* key_right_ptr = reinterpret_cast(rows_right + begin_right); int32_t j; - // length can be zero + // length is greater than zero for (j = 0; j < (static_cast(length) + 31) / 32 - 1; ++j) { __m256i key_left = _mm256_loadu_si256(key_left_ptr + j); __m256i key_right = _mm256_loadu_si256(key_right_ptr + j); @@ -569,6 +571,15 @@ uint32_t KeyCompare::NullUpdateColumnToRow_avx2( const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector) { + int64_t num_rows_safe = + TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), col.bit_offset(0)); + if (sel_left_maybe_null) { + num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( + num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); + } else { + num_rows_to_compare = static_cast(num_rows_safe); + } + if (use_selection) { return NullUpdateColumnToRowImp_avx2(id_col, num_rows_to_compare, sel_left_maybe_null, left_to_right_map, @@ -585,6 +596,29 @@ uint32_t KeyCompare::CompareBinaryColumnToRow_avx2( const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector) { + uint32_t col_width = col.metadata().fixed_length; + int64_t num_rows_safe = col.length(); + if (col_width == 0) { + // In this case we will access left column memory 4B at a time + num_rows_safe = + TailSkipForSIMD::FixBitAccess(sizeof(uint32_t), col.length(), col.bit_offset(1)); + } else if (col_width == 1 || col_width == 2) { + // In this case we will access left column memory 4B at a time + num_rows_safe = + TailSkipForSIMD::FixBinaryAccess(sizeof(uint32_t), col.length(), col_width); + } else if (col_width != 4 && col_width != 8) { + // In this case we will access left column memory 32B at a time + num_rows_safe = + TailSkipForSIMD::FixBinaryAccess(sizeof(__m256i), col.length(), col_width); + } + if (sel_left_maybe_null) { + num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( + num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); + } else { + num_rows_to_compare = static_cast( + std::min(num_rows_safe, static_cast(num_rows_to_compare))); + } + if (use_selection) { return CompareBinaryColumnToRowImp_avx2(offset_within_row, num_rows_to_compare, sel_left_maybe_null, left_to_right_map, @@ -596,11 +630,20 @@ uint32_t KeyCompare::CompareBinaryColumnToRow_avx2( } } -void KeyCompare::CompareVarBinaryColumnToRow_avx2( +uint32_t KeyCompare::CompareVarBinaryColumnToRow_avx2( bool use_selection, bool is_first_varbinary_col, uint32_t id_varlen_col, uint32_t num_rows_to_compare, const uint16_t* sel_left_maybe_null, const uint32_t* left_to_right_map, LightContext* ctx, const KeyColumnArray& col, const RowTableImpl& rows, uint8_t* match_bytevector) { + int64_t num_rows_safe = + TailSkipForSIMD::FixVarBinaryAccess(sizeof(__m256i), col.length(), col.offsets()); + if (use_selection) { + num_rows_to_compare = static_cast(TailSkipForSIMD::FixSelection( + num_rows_safe, static_cast(num_rows_to_compare), sel_left_maybe_null)); + } else { + num_rows_to_compare = static_cast(num_rows_safe); + } + if (use_selection) { if (is_first_varbinary_col) { CompareVarBinaryColumnToRowImp_avx2( @@ -622,6 +665,8 @@ void KeyCompare::CompareVarBinaryColumnToRow_avx2( col, rows, match_bytevector); } } + + return num_rows_to_compare; } #endif diff --git a/cpp/src/arrow/compute/row/encode.cc b/cpp/src/arrow/compute/row/encode.cc new file mode 100644 index 00000000000..4beff5940a0 --- /dev/null +++ b/cpp/src/arrow/compute/row/encode.cc @@ -0,0 +1,456 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/row/encode.h" + +#include +#include // std::upper_bound +#include +#include +#include + +#include "arrow/array/util.h" // MakeArrayFromScalar +#include "arrow/compute/exec/key_hash.h" +#include "arrow/compute/exec/util.h" +#include "arrow/compute/row/compare_internal.h" +#include "arrow/util/bit_util.h" +#include "arrow/util/bitmap_ops.h" + +namespace arrow { +namespace compute { + +Status RowTable::InitIfNeeded(MemoryPool* pool, const RowTableMetadata& row_metadata) { + if (is_initialized_) { + return Status::OK(); + } + encoder_.Init(row_metadata.column_metadatas, sizeof(uint64_t), sizeof(uint64_t)); + RETURN_NOT_OK(rows_temp_.Init(pool, row_metadata)); + RETURN_NOT_OK(rows_.Init(pool, row_metadata)); + is_initialized_ = true; + return Status::OK(); +} + +Status RowTable::InitIfNeeded(MemoryPool* pool, const ExecBatch& batch) { + if (is_initialized_) { + return Status::OK(); + } + std::vector column_metadatas; + RETURN_NOT_OK(ColumnMetadatasFromExecBatch(batch, &column_metadatas)); + RowTableMetadata row_metadata; + row_metadata.FromColumnMetadataVector(column_metadatas, sizeof(uint64_t), + sizeof(uint64_t)); + + return InitIfNeeded(pool, row_metadata); +} + +Status RowTable::AppendBatchSelection(MemoryPool* pool, const ExecBatch& batch, + int begin_row_id, int end_row_id, int num_row_ids, + const uint16_t* row_ids, + std::vector& temp_column_arrays) { + RETURN_NOT_OK(InitIfNeeded(pool, batch)); + RETURN_NOT_OK(ColumnArraysFromExecBatch(batch, begin_row_id, end_row_id - begin_row_id, + &temp_column_arrays)); + encoder_.PrepareEncodeSelected( + /*start_row=*/0, end_row_id - begin_row_id, temp_column_arrays); + RETURN_NOT_OK(encoder_.EncodeSelected(&rows_temp_, num_row_ids, row_ids)); + RETURN_NOT_OK(rows_.AppendSelectionFrom(rows_temp_, num_row_ids, nullptr)); + return Status::OK(); +} + +void RowTable::Compare(const ExecBatch& batch, int begin_row_id, int end_row_id, + int num_selected, const uint16_t* batch_selection_maybe_null, + const uint32_t* array_row_ids, uint32_t* out_num_not_equal, + uint16_t* out_not_equal_selection, int64_t hardware_flags, + util::TempVectorStack* temp_stack, + std::vector& temp_column_arrays, + uint8_t* out_match_bitvector_maybe_null) { + Status status = ColumnArraysFromExecBatch( + batch, begin_row_id, end_row_id - begin_row_id, &temp_column_arrays); + ARROW_DCHECK(status.ok()); + + LightContext ctx; + ctx.hardware_flags = hardware_flags; + ctx.stack = temp_stack; + KeyCompare::CompareColumnsToRows( + num_selected, batch_selection_maybe_null, array_row_ids, &ctx, out_num_not_equal, + out_not_equal_selection, temp_column_arrays, rows_, + /*are_cols_in_encoding_order=*/false, out_match_bitvector_maybe_null); +} + +Status RowTable::DecodeSelected(ResizableArrayData* output, int column_id, + int num_rows_to_append, const uint32_t* row_ids, + MemoryPool* pool) const { + int num_rows_before = output->num_rows(); + RETURN_NOT_OK(output->ResizeFixedLengthBuffers(num_rows_before + num_rows_to_append)); + + // Both input (KeyRowArray) and output (ResizableArrayData) have buffers with + // extra bytes added at the end to avoid buffer overruns when using wide load + // instructions. + // + + ARROW_ASSIGN_OR_RAISE(KeyColumnMetadata column_metadata, output->column_metadata()); + + if (column_metadata.is_fixed_length) { + uint32_t fixed_length = column_metadata.fixed_length; + switch (fixed_length) { + case 0: + RowTableAccessor::Visit(rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + bit_util::SetBitTo(output->mutable_data(1), + num_rows_before + i, *ptr != 0); + }); + break; + case 1: + RowTableAccessor::Visit(rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + output->mutable_data(1)[num_rows_before + i] = *ptr; + }); + break; + case 2: + RowTableAccessor::Visit( + rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + reinterpret_cast(output->mutable_data(1))[num_rows_before + i] = + *reinterpret_cast(ptr); + }); + break; + case 4: + RowTableAccessor::Visit( + rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + reinterpret_cast(output->mutable_data(1))[num_rows_before + i] = + *reinterpret_cast(ptr); + }); + break; + case 8: + RowTableAccessor::Visit( + rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + reinterpret_cast(output->mutable_data(1))[num_rows_before + i] = + *reinterpret_cast(ptr); + }); + break; + default: + RowTableAccessor::Visit( + rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + uint64_t* dst = reinterpret_cast( + output->mutable_data(1) + num_bytes * (num_rows_before + i)); + const uint64_t* src = reinterpret_cast(ptr); + for (uint32_t word_id = 0; + word_id < bit_util::CeilDiv(num_bytes, sizeof(uint64_t)); ++word_id) { + util::SafeStore(dst + word_id, util::SafeLoad(src + word_id)); + } + }); + break; + } + } else { + uint32_t* offsets = + reinterpret_cast(output->mutable_data(1)) + num_rows_before; + uint32_t sum = num_rows_before == 0 ? 0 : offsets[0]; + RowTableAccessor::Visit( + rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { offsets[i] = num_bytes; }); + for (int i = 0; i < num_rows_to_append; ++i) { + uint32_t length = offsets[i]; + offsets[i] = sum; + sum += length; + } + offsets[num_rows_to_append] = sum; + RETURN_NOT_OK(output->ResizeVaryingLengthBuffer()); + RowTableAccessor::Visit( + rows_, column_id, num_rows_to_append, row_ids, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + uint64_t* dst = reinterpret_cast( + output->mutable_data(2) + + reinterpret_cast( + output->mutable_data(1))[num_rows_before + i]); + const uint64_t* src = reinterpret_cast(ptr); + for (uint32_t word_id = 0; + word_id < bit_util::CeilDiv(num_bytes, sizeof(uint64_t)); ++word_id) { + util::SafeStore(dst + word_id, util::SafeLoad(src + word_id)); + } + }); + } + + // Process nulls + // + RowTableAccessor::VisitNulls( + rows_, column_id, num_rows_to_append, row_ids, [&](int i, uint8_t value) { + bit_util::SetBitTo(output->mutable_data(0), num_rows_before + i, value == 0); + }); + + return Status::OK(); +} + +void RowTable::DebugPrintToFile(const char* filename, bool print_sorted) const { + FILE* fout; +#if defined(_MSC_VER) && _MSC_VER >= 1400 + fopen_s(&fout, filename, "wt"); +#else + fout = fopen(filename, "wt"); +#endif + if (!fout) { + return; + } + + for (int64_t row_id = 0; row_id < rows_.length(); ++row_id) { + for (uint32_t column_id = 0; column_id < rows_.metadata().num_cols(); ++column_id) { + bool is_null; + uint32_t row_id_cast = static_cast(row_id); + RowTableAccessor::VisitNulls(rows_, column_id, 1, &row_id_cast, + [&](int i, uint8_t value) { is_null = (value != 0); }); + if (is_null) { + fprintf(fout, "null"); + } else { + RowTableAccessor::Visit(rows_, column_id, 1, &row_id_cast, + [&](int i, const uint8_t* ptr, uint32_t num_bytes) { + fprintf(fout, "\""); + for (uint32_t ibyte = 0; ibyte < num_bytes; ++ibyte) { + fprintf(fout, "%02x", ptr[ibyte]); + } + fprintf(fout, "\""); + }); + } + fprintf(fout, "\t"); + } + fprintf(fout, "\n"); + } + fclose(fout); + + if (print_sorted) { + struct stat sb; + if (stat(filename, &sb) == -1) { + ARROW_DCHECK(false); + return; + } + std::vector buffer; + buffer.resize(sb.st_size); + std::vector lines; + FILE* fin; +#if defined(_MSC_VER) && _MSC_VER >= 1400 + fopen_s(&fin, filename, "rt"); +#else + fin = fopen(filename, "rt"); +#endif + if (!fin) { + return; + } + while (fgets(buffer.data(), static_cast(buffer.size()), fin)) { + lines.push_back(std::string(buffer.data())); + } + fclose(fin); + std::sort(lines.begin(), lines.end()); + FILE* fout2; +#if defined(_MSC_VER) && _MSC_VER >= 1400 + fopen_s(&fout2, filename, "wt"); +#else + fout2 = fopen(filename, "wt"); +#endif + if (!fout2) { + return; + } + for (size_t i = 0; i < lines.size(); ++i) { + fprintf(fout2, "%s\n", lines[i].c_str()); + } + fclose(fout2); + } +} + +Status RowTableMerge::PrepareForMerge(RowTable* target, + const std::vector& sources, + std::vector* first_target_row_id, + MemoryPool* pool) { + ARROW_DCHECK(!sources.empty()); + + ARROW_DCHECK(sources[0]->is_initialized_); + const RowTableMetadata& metadata = sources[0]->rows_.metadata(); + ARROW_DCHECK(!target->is_initialized_); + RETURN_NOT_OK(target->InitIfNeeded(pool, metadata)); + + // Sum the number of rows from all input sources and calculate their total + // size. + // + int64_t num_rows = 0; + int64_t num_bytes = 0; + first_target_row_id->resize(sources.size() + 1); + for (size_t i = 0; i < sources.size(); ++i) { + // All input sources must be initialized and have the same row format. + // + ARROW_DCHECK(sources[i]->is_initialized_); + ARROW_DCHECK(metadata.is_compatible(sources[i]->rows_.metadata())); + (*first_target_row_id)[i] = num_rows; + num_rows += sources[i]->rows_.length(); + if (!metadata.is_fixed_length) { + num_bytes += sources[i]->rows_.offsets()[sources[i]->rows_.length()]; + } + } + (*first_target_row_id)[sources.size()] = num_rows; + + // Allocate target memory + // + target->rows_.Clean(); + RETURN_NOT_OK(target->rows_.AppendEmpty(static_cast(num_rows), + static_cast(num_bytes))); + + // In case of varying length rows, + // initialize the first row offset for each range of rows corresponding to a + // single source. + // + if (!metadata.is_fixed_length) { + num_rows = 0; + num_bytes = 0; + for (size_t i = 0; i < sources.size(); ++i) { + target->rows_.mutable_offsets()[num_rows] = static_cast(num_bytes); + num_rows += sources[i]->rows_.length(); + num_bytes += sources[i]->rows_.offsets()[sources[i]->rows_.length()]; + } + target->rows_.mutable_offsets()[num_rows] = static_cast(num_bytes); + } + + return Status::OK(); +} + +void RowTableMerge::MergeSingle(RowTable* target, const RowTable& source, + int64_t first_target_row_id, + const int64_t* source_rows_permutation) { + // Source and target must: + // - be initialized + // - use the same row format + // - use 64-bit alignment + // + ARROW_DCHECK(source.is_initialized_ && target->is_initialized_); + ARROW_DCHECK(target->rows_.metadata().is_compatible(source.rows_.metadata())); + ARROW_DCHECK(target->rows_.metadata().row_alignment == sizeof(uint64_t)); + + if (target->rows_.metadata().is_fixed_length) { + CopyFixedLength(&target->rows_, source.rows_, first_target_row_id, + source_rows_permutation); + } else { + CopyVaryingLength(&target->rows_, source.rows_, first_target_row_id, + target->rows_.offsets()[first_target_row_id], + source_rows_permutation); + } + CopyNulls(&target->rows_, source.rows_, first_target_row_id, source_rows_permutation); +} + +void RowTableMerge::CopyFixedLength(RowTableImpl* target, const RowTableImpl& source, + int64_t first_target_row_id, + const int64_t* source_rows_permutation) { + int64_t num_source_rows = source.length(); + + int64_t fixed_length = target->metadata().fixed_length; + + // Permutation of source rows is optional. Without permutation all that is + // needed is memcpy. + // + if (!source_rows_permutation) { + memcpy(target->mutable_data(1) + fixed_length * first_target_row_id, source.data(1), + fixed_length * num_source_rows); + } else { + // Row length must be a multiple of 64-bits due to enforced alignment. + // Loop for each output row copying a fixed number of 64-bit words. + // + ARROW_DCHECK(fixed_length % sizeof(uint64_t) == 0); + + int64_t num_words_per_row = fixed_length / sizeof(uint64_t); + for (int64_t i = 0; i < num_source_rows; ++i) { + int64_t source_row_id = source_rows_permutation[i]; + const uint64_t* source_row_ptr = reinterpret_cast( + source.data(1) + fixed_length * source_row_id); + uint64_t* target_row_ptr = reinterpret_cast( + target->mutable_data(1) + fixed_length * (first_target_row_id + i)); + + for (int64_t word = 0; word < num_words_per_row; ++word) { + target_row_ptr[word] = source_row_ptr[word]; + } + } + } +} + +void RowTableMerge::CopyVaryingLength(RowTableImpl* target, const RowTableImpl& source, + int64_t first_target_row_id, + int64_t first_target_row_offset, + const int64_t* source_rows_permutation) { + int64_t num_source_rows = source.length(); + uint32_t* target_offsets = target->mutable_offsets(); + const uint32_t* source_offsets = source.offsets(); + + // Permutation of source rows is optional. + // + if (!source_rows_permutation) { + int64_t target_row_offset = first_target_row_offset; + for (int64_t i = 0; i < num_source_rows; ++i) { + target_offsets[first_target_row_id + i] = static_cast(target_row_offset); + target_row_offset += source_offsets[i + 1] - source_offsets[i]; + } + // We purposefully skip outputting of N+1 offset, to allow concurrent + // copies of rows done to adjacent ranges in target array. + // It should have already been initialized during preparation for merge. + // + + // We can simply memcpy bytes of rows if their order has not changed. + // + memcpy(target->mutable_data(2) + target_offsets[first_target_row_id], source.data(2), + source_offsets[num_source_rows] - source_offsets[0]); + } else { + int64_t target_row_offset = first_target_row_offset; + uint64_t* target_row_ptr = + reinterpret_cast(target->mutable_data(2) + target_row_offset); + for (int64_t i = 0; i < num_source_rows; ++i) { + int64_t source_row_id = source_rows_permutation[i]; + const uint64_t* source_row_ptr = reinterpret_cast( + source.data(2) + source_offsets[source_row_id]); + uint32_t length = source_offsets[source_row_id + 1] - source_offsets[source_row_id]; + + // Rows should be 64-bit aligned. + // In that case we can copy them using a sequence of 64-bit read/writes. + // + ARROW_DCHECK(length % sizeof(uint64_t) == 0); + + for (uint32_t word = 0; word < length / sizeof(uint64_t); ++word) { + *target_row_ptr++ = *source_row_ptr++; + } + + target_offsets[first_target_row_id + i] = static_cast(target_row_offset); + target_row_offset += length; + } + } +} + +void RowTableMerge::CopyNulls(RowTableImpl* target, const RowTableImpl& source, + int64_t first_target_row_id, + const int64_t* source_rows_permutation) { + int64_t num_source_rows = source.length(); + int num_bytes_per_row = target->metadata().null_masks_bytes_per_row; + uint8_t* target_nulls = target->null_masks() + num_bytes_per_row * first_target_row_id; + if (!source_rows_permutation) { + memcpy(target_nulls, source.null_masks(), num_bytes_per_row * num_source_rows); + } else { + for (int64_t i = 0; i < num_source_rows; ++i) { + int64_t source_row_id = source_rows_permutation[i]; + const uint8_t* source_nulls = + source.null_masks() + num_bytes_per_row * source_row_id; + for (int64_t byte = 0; byte < num_bytes_per_row; ++byte) { + *target_nulls++ = *source_nulls++; + } + } + } +} + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/row/encode.h b/cpp/src/arrow/compute/row/encode.h new file mode 100644 index 00000000000..3c5f7e020e9 --- /dev/null +++ b/cpp/src/arrow/compute/row/encode.h @@ -0,0 +1,142 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include "arrow/compute/exec/options.h" +#include "arrow/compute/light_array.h" +#include "arrow/compute/row/encode_internal.h" +#include "arrow/compute/row/row_internal.h" + +namespace arrow { +namespace compute { + +// Write operations (appending batch rows) must not be called by more than one +// thread at the same time. +// +// Read operations (row comparison, column decoding) +// can be called by multiple threads concurrently. +// +struct RowTable { + RowTable() : is_initialized_(false) {} + + Status InitIfNeeded(MemoryPool* pool, const ExecBatch& batch); + Status InitIfNeeded(MemoryPool* pool, const RowTableMetadata& row_metadata); + + Status AppendBatchSelection(MemoryPool* pool, const ExecBatch& batch, int begin_row_id, + int end_row_id, int num_row_ids, const uint16_t* row_ids, + std::vector& temp_column_arrays); + + // This can only be called for a minibatch. + // + void Compare(const ExecBatch& batch, int begin_row_id, int end_row_id, int num_selected, + const uint16_t* batch_selection_maybe_null, const uint32_t* array_row_ids, + uint32_t* out_num_not_equal, uint16_t* out_not_equal_selection, + int64_t hardware_flags, util::TempVectorStack* temp_stack, + std::vector& temp_column_arrays, + uint8_t* out_match_bitvector_maybe_null = NULLPTR); + + // TODO: add AVX2 version + // + Status DecodeSelected(ResizableArrayData* target, int column_id, int num_rows_to_append, + const uint32_t* row_ids, MemoryPool* pool) const; + + void DebugPrintToFile(const char* filename, bool print_sorted) const; + + int64_t num_rows() const { return is_initialized_ ? rows_.length() : 0; } + + bool is_initialized_; + RowTableEncoder encoder_; + RowTableImpl rows_; + RowTableImpl rows_temp_; +}; + +// Implements concatenating multiple row tables into a single one, using +// potentially multiple threads, each processing a single input row array. +// +class RowTableMerge { + public: + // Calculate total number of rows and size in bytes for merged sequence of + // rows and allocate memory for it. + // + // If the rows are of varying length, initialize in the offset array the first + // entry for the write area for each input row array. Leave all other + // offsets and buffers uninitialized. + // + // All input sources must be initialized, but they can contain zero rows. + // + // Output in vector the first target row id for each source (exclusive + // cummulative sum of number of rows in sources). + // + static Status PrepareForMerge(RowTable* target, const std::vector& sources, + std::vector* first_target_row_id, + MemoryPool* pool); + + // Copy rows from source array to target array. + // Both arrays must have the same row metadata. + // Target array must already have the memory reserved in all internal buffers + // for the copy of the rows. + // + // Copy of the rows will occupy the same amount of space in the target array + // buffers as in the source array, but in the target array we pick at what row + // position and offset we start writing. + // + // Optionally, the rows may be reordered during copy according to the + // provided permutation, which represents some sorting order of source rows. + // Nth element of the permutation array is the source row index for the Nth + // row written into target array. If permutation is missing (null), then the + // order of source rows will remain unchanged. + // + // In case of varying length rows, we purposefully skip outputting of N+1 (one + // after last) offset, to allow concurrent copies of rows done to adjacent + // ranges in the target array. This offset should already contain the right + // value after calling the method preparing target array for merge (which + // initializes boundary offsets for target row ranges for each source). + // + static void MergeSingle(RowTable* target, const RowTable& source, + int64_t first_target_row_id, + const int64_t* source_rows_permutation); + + private: + // Copy rows from source array to a region of the target array. + // This implementation is for fixed length rows. + // Null information needs to be handled separately. + // + static void CopyFixedLength(RowTableImpl* target, const RowTableImpl& source, + int64_t first_target_row_id, + const int64_t* source_rows_permutation); + + // Copy rows from source array to a region of the target array. + // This implementation is for varying length rows. + // Null information needs to be handled separately. + // + static void CopyVaryingLength(RowTableImpl* target, const RowTableImpl& source, + int64_t first_target_row_id, + int64_t first_target_row_offset, + const int64_t* source_rows_permutation); + + // Copy null information from rows from source array to a region of the target + // array. + // + static void CopyNulls(RowTableImpl* target, const RowTableImpl& source, + int64_t first_target_row_id, + const int64_t* source_rows_permutation); +}; + +} // namespace compute +} // namespace arrow diff --git a/cpp/src/arrow/compute/row/encode_internal.cc b/cpp/src/arrow/compute/row/encode_internal.cc index cbfd169b448..2664d6e6214 100644 --- a/cpp/src/arrow/compute/row/encode_internal.cc +++ b/cpp/src/arrow/compute/row/encode_internal.cc @@ -20,9 +20,8 @@ namespace arrow { namespace compute { -void RowTableEncoder::Init(const std::vector& cols, LightContext* ctx, - int row_alignment, int string_alignment) { - ctx_ = ctx; +void RowTableEncoder::Init(const std::vector& cols, int row_alignment, + int string_alignment) { row_metadata_.FromColumnMetadataVector(cols, row_alignment, string_alignment); uint32_t num_cols = row_metadata_.num_cols(); uint32_t num_varbinary_cols = row_metadata_.num_varbinary_cols(); @@ -59,18 +58,24 @@ void RowTableEncoder::PrepareKeyColumnArrays(int64_t start_row, int64_t num_rows void RowTableEncoder::DecodeFixedLengthBuffers(int64_t start_row_input, int64_t start_row_output, int64_t num_rows, const RowTableImpl& rows, - std::vector* cols) { + std::vector* cols, + int64_t hardware_flags, + util::TempVectorStack* temp_stack) { // Prepare column array vectors PrepareKeyColumnArrays(start_row_output, num_rows, *cols); + LightContext ctx; + ctx.hardware_flags = hardware_flags; + ctx.stack = temp_stack; + // Create two temp vectors with 16-bit elements auto temp_buffer_holder_A = - util::TempVectorHolder(ctx_->stack, static_cast(num_rows)); + util::TempVectorHolder(ctx.stack, static_cast(num_rows)); auto temp_buffer_A = KeyColumnArray( KeyColumnMetadata(true, sizeof(uint16_t)), num_rows, nullptr, reinterpret_cast(temp_buffer_holder_A.mutable_data()), nullptr); auto temp_buffer_holder_B = - util::TempVectorHolder(ctx_->stack, static_cast(num_rows)); + util::TempVectorHolder(ctx.stack, static_cast(num_rows)); auto temp_buffer_B = KeyColumnArray( KeyColumnMetadata(true, sizeof(uint16_t)), num_rows, nullptr, reinterpret_cast(temp_buffer_holder_B.mutable_data()), nullptr); @@ -79,7 +84,7 @@ void RowTableEncoder::DecodeFixedLengthBuffers(int64_t start_row_input, if (!is_row_fixed_length) { EncoderOffsets::Decode(static_cast(start_row_input), static_cast(num_rows), rows, &batch_varbinary_cols_, - batch_varbinary_cols_base_offsets_, ctx_); + batch_varbinary_cols_base_offsets_, &ctx); } // Process fixed length columns @@ -98,13 +103,13 @@ void RowTableEncoder::DecodeFixedLengthBuffers(int64_t start_row_input, EncoderBinary::Decode(static_cast(start_row_input), static_cast(num_rows), row_metadata_.column_offsets[i], rows, &batch_all_cols_[i], - ctx_, &temp_buffer_A); + &ctx, &temp_buffer_A); i += 1; } else { EncoderBinaryPair::Decode( static_cast(start_row_input), static_cast(num_rows), row_metadata_.column_offsets[i], rows, &batch_all_cols_[i], - &batch_all_cols_[i + 1], ctx_, &temp_buffer_A, &temp_buffer_B); + &batch_all_cols_[i + 1], &ctx, &temp_buffer_A, &temp_buffer_B); i += 2; } } @@ -114,14 +119,17 @@ void RowTableEncoder::DecodeFixedLengthBuffers(int64_t start_row_input, static_cast(num_rows), rows, &batch_all_cols_); } -void RowTableEncoder::DecodeVaryingLengthBuffers(int64_t start_row_input, - int64_t start_row_output, - int64_t num_rows, - const RowTableImpl& rows, - std::vector* cols) { +void RowTableEncoder::DecodeVaryingLengthBuffers( + int64_t start_row_input, int64_t start_row_output, int64_t num_rows, + const RowTableImpl& rows, std::vector* cols, int64_t hardware_flags, + util::TempVectorStack* temp_stack) { // Prepare column array vectors PrepareKeyColumnArrays(start_row_output, num_rows, *cols); + LightContext ctx; + ctx.hardware_flags = hardware_flags; + ctx.stack = temp_stack; + bool is_row_fixed_length = row_metadata_.is_fixed_length; if (!is_row_fixed_length) { for (size_t i = 0; i < batch_varbinary_cols_.size(); ++i) { @@ -129,7 +137,7 @@ void RowTableEncoder::DecodeVaryingLengthBuffers(int64_t start_row_input, // positions in the output row buffer. EncoderVarBinary::Decode(static_cast(start_row_input), static_cast(num_rows), static_cast(i), - rows, &batch_varbinary_cols_[i], ctx_); + rows, &batch_varbinary_cols_[i], &ctx); } } } @@ -878,5 +886,69 @@ void EncoderNulls::EncodeSelected(RowTableImpl* rows, } } +int RowTableAccessor::VarbinaryColumnId(const RowTableMetadata& row_metadata, + int column_id) { + ARROW_DCHECK(row_metadata.num_cols() > static_cast(column_id)); + ARROW_DCHECK(!row_metadata.is_fixed_length); + ARROW_DCHECK(!row_metadata.column_metadatas[column_id].is_fixed_length); + + int varbinary_column_id = 0; + for (int i = 0; i < column_id; ++i) { + if (!row_metadata.column_metadatas[i].is_fixed_length) { + ++varbinary_column_id; + } + } + return varbinary_column_id; +} + +int RowTableAccessor::NumRowsToSkip(const RowTableImpl& rows, int column_id, int num_rows, + const uint32_t* row_ids, int num_tail_bytes_to_skip) { + uint32_t num_bytes_skipped = 0; + int num_rows_left = num_rows; + + bool is_fixed_length_column = + rows.metadata().column_metadatas[column_id].is_fixed_length; + + if (!is_fixed_length_column) { + // Varying length column + // + int varbinary_column_id = VarbinaryColumnId(rows.metadata(), column_id); + + while (num_rows_left > 0 && + num_bytes_skipped < static_cast(num_tail_bytes_to_skip)) { + // Find the pointer to the last requested row + // + uint32_t last_row_id = row_ids[num_rows_left - 1]; + const uint8_t* row_ptr = rows.data(2) + rows.offsets()[last_row_id]; + + // Find the length of the requested varying length field in that row + // + uint32_t field_offset_within_row, field_length; + if (varbinary_column_id == 0) { + rows.metadata().first_varbinary_offset_and_length( + row_ptr, &field_offset_within_row, &field_length); + } else { + rows.metadata().nth_varbinary_offset_and_length( + row_ptr, varbinary_column_id, &field_offset_within_row, &field_length); + } + + num_bytes_skipped += field_length; + --num_rows_left; + } + } else { + // Fixed length column + // + uint32_t field_length = rows.metadata().column_metadatas[column_id].fixed_length; + uint32_t num_bytes_skipped = 0; + while (num_rows_left > 0 && + num_bytes_skipped < static_cast(num_tail_bytes_to_skip)) { + num_bytes_skipped += field_length; + --num_rows_left; + } + } + + return num_rows - num_rows_left; +} + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/row/encode_internal.h b/cpp/src/arrow/compute/row/encode_internal.h index ce887313466..7876986c1c5 100644 --- a/cpp/src/arrow/compute/row/encode_internal.h +++ b/cpp/src/arrow/compute/row/encode_internal.h @@ -46,8 +46,8 @@ namespace compute { /// Does not support nested types class RowTableEncoder { public: - void Init(const std::vector& cols, LightContext* ctx, - int row_alignment, int string_alignment); + void Init(const std::vector& cols, int row_alignment, + int string_alignment); const RowTableMetadata& row_metadata() { return row_metadata_; } // GrouperFastImpl right now needs somewhat intrusive visibility into RowTableEncoder @@ -84,7 +84,8 @@ class RowTableEncoder { /// for the call to DecodeVaryingLengthBuffers void DecodeFixedLengthBuffers(int64_t start_row_input, int64_t start_row_output, int64_t num_rows, const RowTableImpl& rows, - std::vector* cols); + std::vector* cols, int64_t hardware_flags, + util::TempVectorStack* temp_stack); /// \brief Decode the varlength columns of a row table into column storage /// \param start_row_input The starting row to decode @@ -94,7 +95,9 @@ class RowTableEncoder { /// \param cols The column arrays to decode into void DecodeVaryingLengthBuffers(int64_t start_row_input, int64_t start_row_output, int64_t num_rows, const RowTableImpl& rows, - std::vector* cols); + std::vector* cols, + int64_t hardware_flags, + util::TempVectorStack* temp_stack); private: /// Prepare column array vectors. @@ -107,8 +110,6 @@ class RowTableEncoder { void PrepareKeyColumnArrays(int64_t start_row, int64_t num_rows, const std::vector& cols_in); - LightContext* ctx_; - // Data initialized once, based on data types of key columns RowTableMetadata row_metadata_; @@ -319,5 +320,148 @@ class EncoderNulls { std::vector* cols); }; +class RowTableAccessor { + public: + // Find the index of this varbinary column within the sequence of all + // varbinary columns encoded in rows. + // + static int VarbinaryColumnId(const RowTableMetadata& row_metadata, int column_id); + + // Calculate how many rows to skip from the tail of the + // sequence of selected rows, such that the total size of skipped rows is at + // least equal to the size specified by the caller. Skipping of the tail rows + // is used to allow for faster processing by the caller of remaining rows + // without checking buffer bounds (useful with SIMD or fixed size memory loads + // and stores). + // + static int NumRowsToSkip(const RowTableImpl& rows, int column_id, int num_rows, + const uint32_t* row_ids, int num_tail_bytes_to_skip); + + // The supplied lambda will be called for each row in the given list of rows. + // The arguments given to it will be: + // - index of a row (within the set of selected rows), + // - pointer to the value, + // - byte length of the value. + // + // The information about nulls (validity bitmap) is not used in this call and + // has to be processed separately. + // + template + static void Visit(const RowTableImpl& rows, int column_id, int num_rows, + const uint32_t* row_ids, PROCESS_VALUE_FN process_value_fn) { + bool is_fixed_length_column = + rows.metadata().column_metadatas[column_id].is_fixed_length; + + // There are 4 cases, each requiring different steps: + // 1. Varying length column that is the first varying length column in a row + // 2. Varying length column that is not the first varying length column in a + // row + // 3. Fixed length column in a fixed length row + // 4. Fixed length column in a varying length row + + if (!is_fixed_length_column) { + int varbinary_column_id = VarbinaryColumnId(rows.metadata(), column_id); + const uint8_t* row_ptr_base = rows.data(2); + const uint32_t* row_offsets = rows.offsets(); + uint32_t field_offset_within_row, field_length; + + if (varbinary_column_id == 0) { + // Case 1: This is the first varbinary column + // + for (int i = 0; i < num_rows; ++i) { + uint32_t row_id = row_ids[i]; + const uint8_t* row_ptr = row_ptr_base + row_offsets[row_id]; + rows.metadata().first_varbinary_offset_and_length( + row_ptr, &field_offset_within_row, &field_length); + process_value_fn(i, row_ptr + field_offset_within_row, field_length); + } + } else { + // Case 2: This is second or later varbinary column + // + for (int i = 0; i < num_rows; ++i) { + uint32_t row_id = row_ids[i]; + const uint8_t* row_ptr = row_ptr_base + row_offsets[row_id]; + rows.metadata().nth_varbinary_offset_and_length( + row_ptr, varbinary_column_id, &field_offset_within_row, &field_length); + process_value_fn(i, row_ptr + field_offset_within_row, field_length); + } + } + } + + if (is_fixed_length_column) { + uint32_t field_offset_within_row = rows.metadata().encoded_field_offset( + rows.metadata().pos_after_encoding(column_id)); + uint32_t field_length = rows.metadata().column_metadatas[column_id].fixed_length; + // Bit column is encoded as a single byte + // + if (field_length == 0) { + field_length = 1; + } + uint32_t row_length = rows.metadata().fixed_length; + + bool is_fixed_length_row = rows.metadata().is_fixed_length; + if (is_fixed_length_row) { + // Case 3: This is a fixed length column in a fixed length row + // + const uint8_t* row_ptr_base = rows.data(1) + field_offset_within_row; + for (int i = 0; i < num_rows; ++i) { + uint32_t row_id = row_ids[i]; + const uint8_t* row_ptr = row_ptr_base + row_length * row_id; + process_value_fn(i, row_ptr, field_length); + } + } else { + // Case 4: This is a fixed length column in a varying length row + // + const uint8_t* row_ptr_base = rows.data(2) + field_offset_within_row; + const uint32_t* row_offsets = rows.offsets(); + for (int i = 0; i < num_rows; ++i) { + uint32_t row_id = row_ids[i]; + const uint8_t* row_ptr = row_ptr_base + row_offsets[row_id]; + process_value_fn(i, row_ptr, field_length); + } + } + } + } + + // The supplied lambda will be called for each row in the given list of rows. + // The arguments given to it will be: + // - index of a row (within the set of selected rows), + // - byte 0xFF if the null is set for the row or 0x00 otherwise. + // + template + static void VisitNulls(const RowTableImpl& rows, int column_id, int num_rows, + const uint32_t* row_ids, PROCESS_VALUE_FN process_value_fn) { + const uint8_t* null_masks = rows.null_masks(); + uint32_t null_mask_num_bytes = rows.metadata().null_masks_bytes_per_row; + uint32_t pos_after_encoding = rows.metadata().pos_after_encoding(column_id); + for (int i = 0; i < num_rows; ++i) { + uint32_t row_id = row_ids[i]; + int64_t bit_id = row_id * null_mask_num_bytes * 8 + pos_after_encoding; + process_value_fn(i, bit_util::GetBit(null_masks, bit_id) ? 0xff : 0); + } + } + + private: +#if defined(ARROW_HAVE_AVX2) + // This is equivalent to Visit method, but processing 8 rows at a time in a + // loop. + // Returns the number of processed rows, which may be less than requested (up + // to 7 rows at the end may be skipped). + // + template + static int Visit_avx2(const RowTableImpl& rows, int column_id, int num_rows, + const uint32_t* row_ids, PROCESS_8_VALUES_FN process_8_values_fn); + + // This is equivalent to VisitNulls method, but processing 8 rows at a time in + // a loop. Returns the number of processed rows, which may be less than + // requested (up to 7 rows at the end may be skipped). + // + template + static int VisitNulls_avx2(const RowTableImpl& rows, int column_id, int num_rows, + const uint32_t* row_ids, + PROCESS_8_VALUES_FN process_8_values_fn); +#endif +}; + } // namespace compute } // namespace arrow diff --git a/cpp/src/arrow/compute/row/encode_internal_avx2.cc b/cpp/src/arrow/compute/row/encode_internal_avx2.cc index 02ba310bded..8de98e40935 100644 --- a/cpp/src/arrow/compute/row/encode_internal_avx2.cc +++ b/cpp/src/arrow/compute/row/encode_internal_avx2.cc @@ -230,6 +230,172 @@ void EncoderVarBinary::DecodeImp_avx2(uint32_t start_row, uint32_t num_rows, }); } +template +int RowTableAccessor::Visit_avx2(const RowTableImpl& rows, int column_id, int num_rows, + const uint32_t* row_ids, + PROCESS_8_VALUES_FN process_8_values_fn) { + // Number of rows processed together in a single iteration of the loop (single + // call to the provided processing lambda). + // + constexpr int unroll = 8; + + bool is_fixed_length_column = + rows.metadata().column_metadatas[column_id].is_fixed_length; + + // There are 4 cases, each requiring different steps: + // 1. Varying length column that is the first varying length column in a row + // 2. Varying length column that is not the first varying length column in a + // row + // 3. Fixed length column in a fixed length row + // 4. Fixed length column in a varying length row + + if (!is_fixed_length_column) { + int varbinary_column_id = VarbinaryColumnId(rows.metadata(), column_id); + const uint8_t* row_ptr_base = rows.data(2); + const uint32_t* row_offsets = rows.offsets(); + + if (varbinary_column_id == 0) { + // Case 1: This is the first varbinary column + // + __m256i field_offset_within_row = _mm256_set1_epi32(rows.metadata().fixed_length); + __m256i varbinary_end_array_offset = + _mm256_set1_epi32(rows.metadata().varbinary_end_array_offset); + for (int i = 0; i < num_rows / unroll; ++i) { + __m256i row_id = + _mm256_loadu_si256(reinterpret_cast(row_ids) + i); + __m256i row_offset = _mm256_i32gather_epi32( + reinterpret_cast(row_offsets), row_id, sizeof(uint32_t)); + __m256i field_length = _mm256_sub_epi32( + _mm256_i32gather_epi32( + reinterpret_cast(row_ptr_base), + _mm256_add_epi32(row_offset, varbinary_end_array_offset), 1), + field_offset_within_row); + process_8_values_fn(i * unroll, row_ptr_base, + _mm256_add_epi32(row_offset, field_offset_within_row), + field_length); + } + } else { + // Case 2: This is second or later varbinary column + // + __m256i varbinary_end_array_offset = + _mm256_set1_epi32(rows.metadata().varbinary_end_array_offset + + sizeof(uint32_t) * (varbinary_column_id - 1)); + auto row_ptr_base_i64 = + reinterpret_cast(row_ptr_base); + for (int i = 0; i < num_rows / unroll; ++i) { + __m256i row_id = + _mm256_loadu_si256(reinterpret_cast(row_ids) + i); + __m256i row_offset = _mm256_i32gather_epi32( + reinterpret_cast(row_offsets), row_id, sizeof(uint32_t)); + __m256i end_array_offset = + _mm256_add_epi32(row_offset, varbinary_end_array_offset); + + __m256i field_offset_within_row_A = _mm256_i32gather_epi64( + row_ptr_base_i64, _mm256_castsi256_si128(end_array_offset), 1); + __m256i field_offset_within_row_B = _mm256_i32gather_epi64( + row_ptr_base_i64, _mm256_extracti128_si256(end_array_offset, 1), 1); + field_offset_within_row_A = _mm256_permutevar8x32_epi32( + field_offset_within_row_A, _mm256_setr_epi32(0, 2, 4, 6, 1, 3, 5, 7)); + field_offset_within_row_B = _mm256_permutevar8x32_epi32( + field_offset_within_row_B, _mm256_setr_epi32(1, 3, 5, 7, 0, 2, 4, 6)); + + __m256i field_offset_within_row = _mm256_blend_epi32( + field_offset_within_row_A, field_offset_within_row_B, 0xf0); + + __m256i alignment_padding = + _mm256_andnot_si256(field_offset_within_row, _mm256_set1_epi8(0xff)); + alignment_padding = _mm256_add_epi32(alignment_padding, _mm256_set1_epi32(1)); + alignment_padding = _mm256_and_si256( + alignment_padding, _mm256_set1_epi32(rows.metadata().string_alignment - 1)); + + field_offset_within_row = + _mm256_add_epi32(field_offset_within_row, alignment_padding); + + __m256i field_length = _mm256_blend_epi32(field_offset_within_row_A, + field_offset_within_row_B, 0x0f); + field_length = _mm256_permute4x64_epi64(field_length, + 0x4e); // Swapping low and high 128-bits + field_length = _mm256_sub_epi32(field_length, field_offset_within_row); + + process_8_values_fn(i * unroll, row_ptr_base, + _mm256_add_epi32(row_offset, field_offset_within_row), + field_length); + } + } + } + + if (is_fixed_length_column) { + __m256i field_offset_within_row = + _mm256_set1_epi32(rows.metadata().encoded_field_offset( + rows.metadata().pos_after_encoding(column_id))); + __m256i field_length = + _mm256_set1_epi32(rows.metadata().column_metadatas[column_id].fixed_length); + + bool is_fixed_length_row = rows.metadata().is_fixed_length; + if (is_fixed_length_row) { + // Case 3: This is a fixed length column in fixed length row + // + const uint8_t* row_ptr_base = rows.data(1); + for (int i = 0; i < num_rows / unroll; ++i) { + __m256i row_id = + _mm256_loadu_si256(reinterpret_cast(row_ids) + i); + __m256i row_offset = _mm256_mullo_epi32(row_id, field_length); + __m256i field_offset = _mm256_add_epi32(row_offset, field_offset_within_row); + process_8_values_fn(i * unroll, row_ptr_base, field_offset, field_length); + } + } else { + // Case 4: This is a fixed length column in varying length row + // + const uint8_t* row_ptr_base = rows.data(2); + const uint32_t* row_offsets = rows.offsets(); + for (int i = 0; i < num_rows / unroll; ++i) { + __m256i row_id = + _mm256_loadu_si256(reinterpret_cast(row_ids) + i); + __m256i row_offset = _mm256_i32gather_epi32( + reinterpret_cast(row_offsets), row_id, sizeof(uint32_t)); + __m256i field_offset = _mm256_add_epi32(row_offset, field_offset_within_row); + process_8_values_fn(i * unroll, row_ptr_base, field_offset, field_length); + } + } + } + + return num_rows - (num_rows % unroll); +} + +template +int RowTableAccessor::VisitNulls_avx2(const RowTableImpl& rows, int column_id, + int num_rows, const uint32_t* row_ids, + PROCESS_8_VALUES_FN process_8_values_fn) { + // Number of rows processed together in a single iteration of the loop (single + // call to the provided processing lambda). + // + constexpr int unroll = 8; + + const uint8_t* null_masks = rows.null_masks(); + __m256i null_bits_per_row = + _mm256_set1_epi32(8 * rows.metadata().null_masks_bytes_per_row); + for (int i = 0; i < num_rows / unroll; ++i) { + __m256i row_id = _mm256_loadu_si256(reinterpret_cast(row_ids) + i); + __m256i bit_id = _mm256_mullo_epi32(row_id, null_bits_per_row); + bit_id = _mm256_add_epi32(bit_id, _mm256_set1_epi32(column_id)); + __m256i bytes = _mm256_i32gather_epi32(reinterpret_cast(null_masks), + _mm256_srli_epi32(bit_id, 3), 1); + __m256i bit_in_word = _mm256_sllv_epi32( + _mm256_set1_epi32(1), _mm256_and_si256(bit_id, _mm256_set1_epi32(7))); + __m256i result = + _mm256_cmpeq_epi32(_mm256_and_si256(bytes, bit_in_word), bit_in_word); + uint64_t null_bytes = static_cast( + _mm256_movemask_epi8(_mm256_cvtepi32_epi64(_mm256_castsi256_si128(result)))); + null_bytes |= static_cast(_mm256_movemask_epi8( + _mm256_cvtepi32_epi64(_mm256_extracti128_si256(result, 1)))) + << 32; + + process_8_values_fn(i * unroll, null_bytes); + } + + return num_rows - (num_rows % unroll); +} + #endif } // namespace compute diff --git a/cpp/src/arrow/compute/row/grouper.cc b/cpp/src/arrow/compute/row/grouper.cc index eb55124b179..ece0a2d9fbc 100644 --- a/cpp/src/arrow/compute/row/grouper.cc +++ b/cpp/src/arrow/compute/row/grouper.cc @@ -235,7 +235,7 @@ struct GrouperFastImpl : Grouper { impl->key_types_[icol] = key; } - impl->encoder_.Init(impl->col_metadata_, &impl->encode_ctx_, + impl->encoder_.Init(impl->col_metadata_, /* row_alignment = */ sizeof(uint64_t), /* string_alignment = */ sizeof(uint64_t)); RETURN_NOT_OK(impl->rows_.Init(ctx->memory_pool(), impl->encoder_.row_metadata())); @@ -250,7 +250,8 @@ struct GrouperFastImpl : Grouper { KeyCompare::CompareColumnsToRows( num_keys_to_compare, selection_may_be_null, group_ids, &impl_ptr->encode_ctx_, out_num_keys_mismatch, out_selection_mismatch, - impl_ptr->encoder_.batch_all_cols(), impl_ptr->rows_); + impl_ptr->encoder_.batch_all_cols(), impl_ptr->rows_, + /*are_cols_in_encoding_order=*/true); }; auto append_func = [impl_ptr](int num_keys, const uint16_t* selection) { RETURN_NOT_OK(impl_ptr->encoder_.EncodeSelected(&impl_ptr->rows_minibatch_, @@ -438,7 +439,7 @@ struct GrouperFastImpl : Grouper { int64_t batch_size_next = std::min(num_groups - start_row, static_cast(minibatch_size_max_)); encoder_.DecodeFixedLengthBuffers(start_row, start_row, batch_size_next, rows_, - &cols_); + &cols_, encode_ctx_.hardware_flags, &temp_stack_); start_row += batch_size_next; } @@ -458,7 +459,8 @@ struct GrouperFastImpl : Grouper { int64_t batch_size_next = std::min(num_groups - start_row, static_cast(minibatch_size_max_)); encoder_.DecodeVaryingLengthBuffers(start_row, start_row, batch_size_next, rows_, - &cols_); + &cols_, encode_ctx_.hardware_flags, + &temp_stack_); start_row += batch_size_next; } } diff --git a/cpp/src/arrow/compute/row/row_internal.cc b/cpp/src/arrow/compute/row/row_internal.cc index c4e9cc99af5..428befcc7ec 100644 --- a/cpp/src/arrow/compute/row/row_internal.cc +++ b/cpp/src/arrow/compute/row/row_internal.cc @@ -111,6 +111,11 @@ void RowTableMetadata::FromColumnMetadataVector( return left < right; }); + inverse_column_order.resize(num_cols); + for (uint32_t i = 0; i < num_cols; ++i) { + inverse_column_order[column_order[i]] = i; + } + row_alignment = in_row_alignment; string_alignment = in_string_alignment; varbinary_end_array_offset = 0; diff --git a/cpp/src/arrow/compute/row/row_internal.h b/cpp/src/arrow/compute/row/row_internal.h index 6b53c72b45c..d8cb69cb337 100644 --- a/cpp/src/arrow/compute/row/row_internal.h +++ b/cpp/src/arrow/compute/row/row_internal.h @@ -73,6 +73,7 @@ struct ARROW_EXPORT RowTableMetadata { /// Order in which fields are encoded. std::vector column_order; + std::vector inverse_column_order; /// Offsets within a row to fields in their encoding order. std::vector column_offsets; @@ -133,6 +134,8 @@ struct ARROW_EXPORT RowTableMetadata { uint32_t encoded_field_order(uint32_t icol) const { return column_order[icol]; } + uint32_t pos_after_encoding(uint32_t icol) const { return inverse_column_order[icol]; } + uint32_t encoded_field_offset(uint32_t icol) const { return column_offsets[icol]; } uint32_t num_cols() const { return static_cast(column_metadatas.size()); } From 597b7c05e7aae4cce2b16c730244577e4ddcf329 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Mon, 23 May 2022 12:25:56 -1000 Subject: [PATCH 3/3] ARROW-16590: Add unit test for RowTable --- cpp/src/arrow/compute/exec/test_util.cc | 8 +- cpp/src/arrow/compute/exec/test_util.h | 3 +- cpp/src/arrow/compute/row/CMakeLists.txt | 2 + cpp/src/arrow/compute/row/encode_test.cc | 303 +++++++++++++++++++++++ 4 files changed, 312 insertions(+), 4 deletions(-) create mode 100644 cpp/src/arrow/compute/row/encode_test.cc diff --git a/cpp/src/arrow/compute/exec/test_util.cc b/cpp/src/arrow/compute/exec/test_util.cc index 3f5d094774c..3f13387dd45 100644 --- a/cpp/src/arrow/compute/exec/test_util.cc +++ b/cpp/src/arrow/compute/exec/test_util.cc @@ -211,7 +211,7 @@ BatchesWithSchema MakeNestedBatches() { } BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, - int num_batches, int batch_size) { + int num_batches, int batch_size, bool add_tag) { BatchesWithSchema out; random::RandomArrayGenerator rng(42); @@ -219,8 +219,10 @@ BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, for (int i = 0; i < num_batches; ++i) { out.batches[i] = ExecBatch(*rng.BatchOf(schema->fields(), batch_size)); - // add a tag scalar to ensure the batches are unique - out.batches[i].values.emplace_back(i); + if (add_tag) { + // add a tag scalar to ensure the batches are unique + out.batches[i].values.emplace_back(i); + } } out.schema = schema; diff --git a/cpp/src/arrow/compute/exec/test_util.h b/cpp/src/arrow/compute/exec/test_util.h index 9cb615ac450..6e4ede2e82c 100644 --- a/cpp/src/arrow/compute/exec/test_util.h +++ b/cpp/src/arrow/compute/exec/test_util.h @@ -97,7 +97,8 @@ BatchesWithSchema MakeNestedBatches(); ARROW_TESTING_EXPORT BatchesWithSchema MakeRandomBatches(const std::shared_ptr& schema, - int num_batches = 10, int batch_size = 4); + int num_batches = 10, int batch_size = 4, + bool add_tag = true); ARROW_TESTING_EXPORT Result> SortTableOnAllFields(const std::shared_ptr& tab); diff --git a/cpp/src/arrow/compute/row/CMakeLists.txt b/cpp/src/arrow/compute/row/CMakeLists.txt index 6ae982dbaf3..648c65c4ab6 100644 --- a/cpp/src/arrow/compute/row/CMakeLists.txt +++ b/cpp/src/arrow/compute/row/CMakeLists.txt @@ -19,3 +19,5 @@ # in a row-major order. arrow_install_all_headers("arrow/compute/row") + +add_arrow_compute_test(row_test SOURCES encode_test.cc) diff --git a/cpp/src/arrow/compute/row/encode_test.cc b/cpp/src/arrow/compute/row/encode_test.cc new file mode 100644 index 00000000000..1a58261f633 --- /dev/null +++ b/cpp/src/arrow/compute/row/encode_test.cc @@ -0,0 +1,303 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "arrow/compute/row/encode.h" + +#include +#include +#include + +#include + +#include "arrow/array/builder_primitive.h" +#include "arrow/array/concatenate.h" +#include "arrow/array/util.h" +#include "arrow/compute/api_vector.h" +#include "arrow/compute/exec/test_util.h" +#include "arrow/compute/light_array.h" +#include "arrow/type.h" +#include "arrow/util/bit_util.h" +#include "arrow/util/macros.h" + +namespace arrow { +namespace compute { + +void CheckRowTable(const std::vector& batches, const RowTable& row_table, + const std::vector& row_ids) { + ASSERT_FALSE(batches.empty()); + int num_rows = 0; + for (const auto& batch : batches) { + num_rows += static_cast(batch.length); + } + int num_columns = batches[0].num_values(); + ASSERT_LE(num_rows, row_table.num_rows()); + for (int i = 0; i < num_columns; i++) { + ResizableArrayData target; + target.Init(batches[0].values[i].type(), default_memory_pool(), + bit_util::Log2(num_rows)); + ASSERT_OK(row_table.DecodeSelected(&target, i, num_rows, row_ids.data(), + default_memory_pool())); + std::shared_ptr decoded = target.array_data(); + std::vector> expected_arrays; + for (const auto& batch : batches) { + expected_arrays.push_back(batch.values[i].make_array()); + } + ASSERT_OK_AND_ASSIGN(std::shared_ptr expected_array, + Concatenate(expected_arrays)); + AssertArraysEqual(*expected_array, *MakeArray(decoded)); + } +} + +Result EncodeEntireBatches(RowTable* row_table, + const std::vector& batches) { + std::vector temp_column_arrays; + int num_rows_encoded = 0; + for (const auto& batch : batches) { + int num_rows = static_cast(batch.length); + std::vector in_row_ids(num_rows); + std::iota(in_row_ids.begin(), in_row_ids.end(), 0); + RETURN_NOT_OK(row_table->InitIfNeeded(default_memory_pool(), batch)); + RETURN_NOT_OK(row_table->AppendBatchSelection(default_memory_pool(), batch, 0, + num_rows, num_rows, in_row_ids.data(), + temp_column_arrays)); + num_rows_encoded += num_rows; + } + return num_rows_encoded; +} + +void CheckRoundTrip(const ExecBatch& batch, std::vector row_ids = {}) { + if (row_ids.empty()) { + row_ids.resize(batch.length); + std::iota(row_ids.begin(), row_ids.end(), 0); + } + std::vector array_row_ids(row_ids.size()); + std::iota(array_row_ids.begin(), array_row_ids.end(), 0); + + RowTable row_table; + ASSERT_OK(EncodeEntireBatches(&row_table, {batch})); + CheckRowTable({batch}, row_table, array_row_ids); +} + +TEST(RowTable, Basic) { + // All fixed + CheckRoundTrip(ExecBatchFromJSON({int32(), int32(), int32()}, R"([ + [1, 4, 7], + [2, 5, 8], + [3, 6, 9] + ])")); + // All varlen + CheckRoundTrip(ExecBatchFromJSON({utf8(), utf8()}, R"([ + ["xyz", "longer string"], + ["really even longer string", ""], + [null, "a"], + ["null", null], + ["b", "c"] + ])")); + // Mixed types + CheckRoundTrip(ExecBatchFromJSON({boolean(), int32(), utf8()}, R"([ + [true, 3, "test"], + [null, null, null], + [false, 0, "blah"] + ])")); + // No nulls + CheckRoundTrip(ExecBatchFromJSON({int16(), utf8()}, R"([ + [10, "blahblah"], + [8, "not-null"] + ])")); +} + +TEST(RowTable, MultiBatch) { + constexpr int kRowsPerBatch = std::numeric_limits::max(); + constexpr int num_batches = 4; + constexpr int num_out_rows = kRowsPerBatch * num_batches; + std::vector in_row_ids(kRowsPerBatch); + std::vector out_row_ids(num_out_rows); + std::iota(in_row_ids.begin(), in_row_ids.end(), 0); + std::iota(out_row_ids.begin(), out_row_ids.end(), 0); + // Should be able to encode multiple batches to something + // greater than 2^16 rows + BatchesWithSchema test_data = MakeRandomBatches( + schema({field("bool", boolean()), field("i8", int8()), field("utf8", utf8())}), 4, + kRowsPerBatch, /*add_tag=*/false); + + RowTable row_table; + ASSERT_OK_AND_ASSIGN(int num_rows_encoded, + EncodeEntireBatches(&row_table, test_data.batches)); + ASSERT_EQ(num_rows_encoded, row_table.num_rows()); + + CheckRowTable(test_data.batches, row_table, out_row_ids); +} + +Result ColumnTake(const ExecBatch& input, + const std::vector& row_ids) { + UInt16Builder take_indices_builder; + RETURN_NOT_OK(take_indices_builder.AppendValues(row_ids)); + ARROW_ASSIGN_OR_RAISE(std::shared_ptr take_indices, + take_indices_builder.Finish()); + + std::vector taken_arrays(input.num_values()); + for (int i = 0; i < input.num_values(); i++) { + ARROW_ASSIGN_OR_RAISE(Datum taken_array, Take(input.values[i], take_indices)); + taken_arrays[i] = taken_array; + } + return ExecBatch(taken_arrays, static_cast(row_ids.size())); +} + +TEST(RowTable, InputPartialSelection) { + constexpr int num_rows = std::numeric_limits::max(); + BatchesWithSchema test_data = MakeRandomBatches( + schema({field("bool", boolean()), field("i8", int8()), field("utf8", utf8())}), 1, + num_rows, /*add_tag=*/false); + ExecBatch batch = test_data.batches[0]; + std::vector row_ids; + std::default_random_engine gen(42); + std::uniform_int_distribution<> dist(0, 1); + for (uint16_t i = 0; i < num_rows; i++) { + if (dist(gen)) { + row_ids.push_back(i); + } + } + int num_selected_rows = static_cast(row_ids.size()); + ASSERT_GT(num_selected_rows, 0); + ASSERT_LT(num_selected_rows, num_rows); + + std::vector array_row_ids(row_ids.size()); + std::iota(array_row_ids.begin(), array_row_ids.end(), 0); + + RowTable row_table; + std::vector temp_column_arrays; + ASSERT_OK(row_table.InitIfNeeded(default_memory_pool(), batch)); + ASSERT_OK(row_table.AppendBatchSelection(default_memory_pool(), batch, 0, + num_selected_rows, num_selected_rows, + row_ids.data(), temp_column_arrays)); + + ASSERT_OK_AND_ASSIGN(ExecBatch expected, ColumnTake(batch, row_ids)); + CheckRowTable({expected}, row_table, array_row_ids); +} + +TEST(RowTable, ThreadedDecode) { + // Create kBatchesPerThread batches per thread. Encode all batches from the main + // thread. Then decode in parallel so each thread decodes it's own batches + constexpr int kRowsPerBatch = std::numeric_limits::max(); + constexpr int kNumThreads = 16; + constexpr int kBatchesPerThread = 2; + constexpr int kNumBatches = kNumThreads * kBatchesPerThread; + constexpr int kNumOutRows = kRowsPerBatch * kBatchesPerThread; + + BatchesWithSchema test_data = MakeRandomBatches( + schema({field("bool", boolean()), field("i8", int8()), field("utf8", utf8())}), + kNumBatches, kRowsPerBatch, /*add_tag=*/false); + + RowTable row_table; + ASSERT_OK_AND_ASSIGN(int num_rows_encoded, + EncodeEntireBatches(&row_table, test_data.batches)); + ASSERT_EQ(num_rows_encoded, kNumOutRows * kNumThreads); + + std::vector> row_ids_for_threads(kNumThreads); + uint32_t row_id_offset = 0; + for (int i = 0; i < kNumThreads; i++) { + std::vector& row_ids_for_thread = row_ids_for_threads[i]; + row_ids_for_thread.resize(kNumOutRows); + std::iota(row_ids_for_thread.begin(), row_ids_for_thread.end(), row_id_offset); + row_id_offset += kNumOutRows; + } + + std::vector thread_tasks; + for (int i = 0; i < kNumThreads; i++) { + thread_tasks.emplace_back([&, i] { + CheckRowTable({test_data.batches[i * 2], test_data.batches[i * 2 + 1]}, row_table, + row_ids_for_threads[i]); + }); + } + for (auto& thread_task : thread_tasks) { + thread_task.join(); + } +} + +void CheckComparison(const ExecBatch& left, const ExecBatch& right, + const std::vector expected) { + RowTable row_table; + ASSERT_OK(EncodeEntireBatches(&row_table, {left})); + std::vector row_ids(right.length); + std::iota(row_ids.begin(), row_ids.end(), 0); + uint32_t num_not_equal = 0; + std::vector not_equal_selection(right.length); + LightContext light_context; + light_context.hardware_flags = + arrow::internal::CpuInfo::GetInstance()->hardware_flags(); + util::TempVectorStack temp_stack; + ASSERT_OK(temp_stack.Init(default_memory_pool(), + 4 * util::MiniBatch::kMiniBatchLength * sizeof(uint32_t))); + light_context.stack = &temp_stack; + std::vector temp_column_arrays; + std::vector match_bit_vector(bit_util::Log2(right.length)); + row_table.Compare( + right, 0, static_cast(right.length), static_cast(right.length), nullptr, + row_ids.data(), &num_not_equal, not_equal_selection.data(), + light_context.hardware_flags, light_context.stack, temp_column_arrays, nullptr); + + // We over-allocated above + not_equal_selection.resize(num_not_equal); + std::vector expected_not_equal_selection; + for (uint16_t i = 0; i < static_cast(expected.size()); i++) { + if (!expected[i]) { + expected_not_equal_selection.push_back(i); + } + } + ASSERT_EQ(expected_not_equal_selection, not_equal_selection); + + int expected_not_equal = 0; + for (bool val : expected) { + if (!val) { + expected_not_equal++; + } + } + ASSERT_EQ(expected_not_equal, num_not_equal); +} + +TEST(RowTable, Compare) { + ExecBatch left = ExecBatchFromJSON({int16(), utf8()}, R"([ + [1, "blahblah"], + [2, null], + [3, "xyz"], + [4, "sample"], + [null, "5"], + [null, null] + ])"); + ExecBatch right = ExecBatchFromJSON({int16(), utf8()}, R"([ + [1, "blahblah"], + [2, "not-null"], + [3, "abc"], + [5, "blah"], + [null, "5"], + [null, null] + ])"); + std::vector expected_matches = { + true, // Matches exactly without nulls + false, // differs by null only + false, // differs by one value + false, // differs by all values + true, // equal but has nulls + true // equal-all-null + }; + // Test in both directions + CheckComparison(left, right, expected_matches); + CheckComparison(right, left, expected_matches); +} + +} // namespace compute +} // namespace arrow