From 0557d4176fca272ac98d644bbb3dd8fa87333d7a Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 11 Apr 2022 12:17:55 -0700 Subject: [PATCH 01/81] squashed with bracket/brace test --- cpp/src/io/fst/agent_dfa.cuh | 722 +++++++++++++++++++++++++++ cpp/src/io/fst/device_dfa.cuh | 264 ++++++++++ cpp/src/io/fst/dispatch_dfa.cuh | 462 +++++++++++++++++ cpp/src/io/fst/in_reg_array.cuh | 138 +++++ cpp/src/io/fst/symbol_lut.cuh | 182 +++++++ cpp/src/io/fst/transition_table.cuh | 149 ++++++ cpp/src/io/fst/translation_table.cuh | 200 ++++++++ cpp/tests/CMakeLists.txt | 1 + cpp/tests/io/fst/fst_test.cu | 291 +++++++++++ 9 files changed, 2409 insertions(+) create mode 100644 cpp/src/io/fst/agent_dfa.cuh create mode 100644 cpp/src/io/fst/device_dfa.cuh create mode 100644 cpp/src/io/fst/dispatch_dfa.cuh create mode 100644 cpp/src/io/fst/in_reg_array.cuh create mode 100644 cpp/src/io/fst/symbol_lut.cuh create mode 100644 cpp/src/io/fst/transition_table.cuh create mode 100644 cpp/src/io/fst/translation_table.cuh create mode 100644 cpp/tests/io/fst/fst_test.cu diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh new file mode 100644 index 00000000000..d983f9287a9 --- /dev/null +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -0,0 +1,722 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 "in_reg_array.cuh" + +#include <cudf/types.hpp> + +#include <cub/cub.cuh> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +//----------------------------------------------------------------------------- +// STATE VECTOR +//----------------------------------------------------------------------------- +/** + * @brief A vector is able to hold multiple state indices (e.g., to represent multiple DFA + * instances, where the i-th item would represent the i-th DFA instance). + * + * @tparam StateIndexT Signed or unsigned type used to index items inside the vector + * @tparam NUM_ITEMS The number of items to be allocated for a vector + */ +template <typename StateIndexT, int32_t NUM_ITEMS> +class MultiItemStateVector { + public: + template <typename IndexT> + constexpr CUDF_HOST_DEVICE void Set(IndexT index, StateIndexT value) noexcept + { + state_[index] = value; + } + + template <typename IndexT> + constexpr CUDF_HOST_DEVICE StateIndexT Get(IndexT index) const noexcept + { + return state_[index]; + } + + private: + StateIndexT state_[NUM_ITEMS]; +}; + +//----------------------------------------------------------------------------- +// DFA-SIMULATION STATE COMPOSITION FUNCTORS +//----------------------------------------------------------------------------- +/** + * @brief Implements an associative composition operation for state transition vectors and + * offset-to-overap vectors to be used with a prefix scan. + * + * l r = c ( s->l->r) + * 0: [2] [1] [2] (i.e. 0->2->2) + * 1: [1] [2] [2] (i.e. 1->1->2) + * 2: [0] [2] [1] (i.e. 2->0->2) + * @tparam NUM_ITEMS The number of items stored within a vector + */ +template <int32_t NUM_ITEMS> +struct VectorCompositeOp { + template <typename VectorT> + constexpr CUDF_HOST_DEVICE VectorT operator()(VectorT const& lhs, VectorT const& rhs) + { + VectorT res; + for (int32_t i = 0; i < NUM_ITEMS; ++i) { + res.Set(i, rhs.Get(lhs.Get(i))); + } + return res; + } +}; + +//----------------------------------------------------------------------------- +// DFA-SIMULATION CALLBACK WRAPPERS/HELPERS +//----------------------------------------------------------------------------- +template <typename TransducerTableT, typename TransducedOutItT, typename TransducedIndexOutItT> +class DFASimulationCallbackWrapper { + public: + __host__ __device__ __forceinline__ DFASimulationCallbackWrapper( + TransducerTableT transducer_table, TransducedOutItT out_it, TransducedIndexOutItT out_idx_it) + : transducer_table(transducer_table), out_it(out_it), out_idx_it(out_idx_it), write(false) + { + } + + template <typename OffsetT> + __host__ __device__ __forceinline__ void Init(OffsetT const& offset) + { + this->offset = offset; + if (!write) out_count = 0; + } + + template <typename CharIndexT, typename StateVectorT, typename SymbolIndexT> + __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const& character_index, + StateVectorT const& old_state, + StateVectorT const& new_state, + SymbolIndexT const& symbol_id) + { + uint32_t count = transducer_table(old_state.Get(0), symbol_id); + if (write) { + for (uint32_t out_char = 0; out_char < count; out_char++) { + out_it[out_count + out_char] = + transducer_table(old_state.Get(0), symbol_id, out_char); + out_idx_it[out_count + out_char] = offset + character_index; + } + } + out_count += count; + } + + __host__ __device__ __forceinline__ void TearDown() {} + + public: + TransducerTableT transducer_table; + TransducedOutItT out_it; + TransducedIndexOutItT out_idx_it; + uint32_t out_count; + uint32_t offset; + bool write; +}; + +//----------------------------------------------------------------------------- +// STATE-TRANSITION CALLBACKS +//----------------------------------------------------------------------------- +class StateTransitionCallbackOp { + public: + template <typename CharIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const& character_index, + SymbolIndexT const& read_symbol_id) const + { + } +}; +/// Type alias for a state transition callback class that performs no operation on any callback +using NoOpStateTransitionOp = StateTransitionCallbackOp; + +template <int32_t NUM_INSTANCES, typename StateVectorT, typename TransitionTableT> +class StateVectorTransitionOp : public StateTransitionCallbackOp { + public: + __host__ __device__ __forceinline__ + StateVectorTransitionOp(TransitionTableT const& transition_table, StateVectorT& state_vector) + : transition_table(transition_table), state_vector(state_vector) + { + } + + template <typename CharIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const& character_index, + SymbolIndexT const read_symbol_id) const + { + using TransitionVectorT = typename TransitionTableT::TransitionVectorT; + + for (int32_t i = 0; i < NUM_INSTANCES; ++i) { + state_vector.Set(i, transition_table(state_vector.Get(i), read_symbol_id)); + } + } + + public: + StateVectorT& state_vector; + const TransitionTableT& transition_table; +}; + +template <typename CallbackOpT, typename StateVectorT, typename TransitionTableT> +struct StateTransitionOp { + StateVectorT old_state_vector; + StateVectorT state_vector; + const TransitionTableT& transition_table; + CallbackOpT& callback_op; + + __host__ __device__ __forceinline__ StateTransitionOp(const TransitionTableT& transition_table, + StateVectorT state_vector, + CallbackOpT& callback_op) + : transition_table(transition_table), + state_vector(state_vector), + old_state_vector(state_vector), + callback_op(callback_op) + { + } + + template <typename CharIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ void ReadSymbol(const CharIndexT& character_index, + const SymbolIndexT& read_symbol_id) + { + using TransitionVectorT= typename TransitionTableT::TransitionVectorT ; + old_state_vector = state_vector; + state_vector.Set(0, transition_table(state_vector.Get(0), read_symbol_id)); + callback_op.ReadSymbol(character_index, old_state_vector, state_vector, read_symbol_id); + } +}; + +template <typename AgentDFAPolicy, typename SymbolItT, typename OffsetT> +struct AgentDFA { + using SymbolIndexT = uint32_t; + using StateIndexT = uint32_t; + using AliasedLoadT = uint32_t; + using CharT = typename std::iterator_traits<SymbolItT>::value_type; + + //------------------------------------------------------------------------------ + // DERIVED CONFIGS + //------------------------------------------------------------------------------ + static constexpr uint32_t BLOCK_THREADS = AgentDFAPolicy::BLOCK_THREADS; + static constexpr uint32_t ITEMS_PER_THREAD = AgentDFAPolicy::ITEMS_PER_THREAD; + + // The number of symbols per thread + static constexpr uint32_t SYMBOLS_PER_THREAD = ITEMS_PER_THREAD; + static constexpr uint32_t SYMBOLS_PER_BLOCK = BLOCK_THREADS * SYMBOLS_PER_THREAD; + + static constexpr uint32_t MIN_UINTS_PER_BLOCK = + CUB_QUOTIENT_CEILING(SYMBOLS_PER_BLOCK, sizeof(AliasedLoadT)); + static constexpr uint32_t UINTS_PER_THREAD = + CUB_QUOTIENT_CEILING(MIN_UINTS_PER_BLOCK, BLOCK_THREADS); + static constexpr uint32_t UINTS_PER_BLOCK = UINTS_PER_THREAD * BLOCK_THREADS; + static constexpr uint32_t SYMBOLS_PER_UINT_BLOCK = UINTS_PER_BLOCK * sizeof(AliasedLoadT); + + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + struct _TempStorage { + // For aliased loading of characters into shared memory + union { + CharT chars[SYMBOLS_PER_BLOCK]; + AliasedLoadT uints[UINTS_PER_BLOCK]; + }; + }; + + struct TempStorage : cub::Uninitialized<_TempStorage> { + }; + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __device__ __forceinline__ AgentDFA(TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()) + { + } + + //--------------------------------------------------------------------- + // STATIC PARSING PRIMITIVES + //--------------------------------------------------------------------- + template <int32_t NUM_SYMBOLS, // The net (excluding overlap) number of characters to be parsed + typename SymbolMatcherT, // The symbol matcher returning the matched symbol and its + // length + typename CallbackOpT, // Callback operator + int32_t IS_FULL_BLOCK> + __device__ __forceinline__ static void ThreadParse(const SymbolMatcherT& symbol_matcher, + const CharT* chars, + const SymbolIndexT& max_num_chars, + CallbackOpT callback_op, + cub::Int2Type<IS_FULL_BLOCK> /*IS_FULL_BLOCK*/) + { + uint32_t matched_id; + + // Iterate over symbols +#pragma unroll + for (int32_t i = 0; i < NUM_SYMBOLS; ++i) { + if (IS_FULL_BLOCK || threadIdx.x * SYMBOLS_PER_THREAD + i < max_num_chars) { + matched_id = symbol_matcher(chars[i]); + callback_op.ReadSymbol(i, matched_id); + } + } + } + + template <int32_t NUM_SYMBOLS, + typename SymbolMatcherT, + typename StateTransitionOpT, + int32_t IS_FULL_BLOCK> + __device__ __forceinline__ void GetThreadStateTransitions( + const SymbolMatcherT& symbol_matcher, + const CharT* chars, + const SymbolIndexT& max_num_chars, + StateTransitionOpT& state_transition_op, + cub::Int2Type<IS_FULL_BLOCK> /*IS_FULL_BLOCK*/) + { + ThreadParse<NUM_SYMBOLS>( + symbol_matcher, chars, max_num_chars, state_transition_op, cub::Int2Type<IS_FULL_BLOCK>()); + } + + //--------------------------------------------------------------------- + // LOADING FULL BLOCK OF CHARACTERS, NON-ALIASED + //--------------------------------------------------------------------- + __device__ __forceinline__ void LoadBlock(const CharT* d_chars, + const OffsetT block_offset, + const OffsetT num_total_symbols, + cub::Int2Type<true> /*IS_FULL_BLOCK*/, + cub::Int2Type<1> /*ALIGNMENT*/) + { + CharT thread_chars[SYMBOLS_PER_THREAD]; + + const CharT* d_block_symbols = d_chars + block_offset; + cub::LoadDirectStriped<BLOCK_THREADS>(threadIdx.x, d_block_symbols, thread_chars); + +#pragma unroll + for (int32_t i = 0; i < SYMBOLS_PER_THREAD; ++i) { + temp_storage.chars[threadIdx.x + i * BLOCK_THREADS] = thread_chars[i]; + } + } + + //--------------------------------------------------------------------- + // LOADING PARTIAL BLOCK OF CHARACTERS, NON-ALIASED + //--------------------------------------------------------------------- + __device__ __forceinline__ void LoadBlock(const CharT* d_chars, + const OffsetT block_offset, + const OffsetT num_total_symbols, + cub::Int2Type<false> /*IS_FULL_BLOCK*/, + cub::Int2Type<1> /*ALIGNMENT*/) + { + CharT thread_chars[SYMBOLS_PER_THREAD]; + + if (num_total_symbols <= block_offset) return; + + // Last unit to be loaded is IDIV_CEIL(#SYM, SYMBOLS_PER_UNIT) + OffsetT num_total_chars = num_total_symbols - block_offset; + + const CharT* d_block_symbols = d_chars + block_offset; + cub::LoadDirectStriped<BLOCK_THREADS>( + threadIdx.x, d_block_symbols, thread_chars, num_total_chars); + +#pragma unroll + for (int32_t i = 0; i < SYMBOLS_PER_THREAD; ++i) { + temp_storage.chars[threadIdx.x + i * BLOCK_THREADS] = thread_chars[i]; + } + } + + //--------------------------------------------------------------------- + // LOADING FULL BLOCK OF CHARACTERS, ALIASED + //--------------------------------------------------------------------- + __device__ __forceinline__ void LoadBlock(const CharT* d_chars, + const OffsetT block_offset, + const OffsetT num_total_symbols, + cub::Int2Type<true> /*IS_FULL_BLOCK*/, + cub::Int2Type<sizeof(AliasedLoadT)> /*ALIGNMENT*/) + { + AliasedLoadT thread_units[UINTS_PER_THREAD]; + + const AliasedLoadT* d_block_symbols = reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); + cub::LoadDirectStriped<BLOCK_THREADS>(threadIdx.x, d_block_symbols, thread_units); + +#pragma unroll + for (int32_t i = 0; i < UINTS_PER_THREAD; ++i) { + temp_storage.uints[threadIdx.x + i * BLOCK_THREADS] = thread_units[i]; + } + } + + //--------------------------------------------------------------------- + // LOADING PARTIAL BLOCK OF CHARACTERS, ALIASED + //--------------------------------------------------------------------- + __device__ __forceinline__ void LoadBlock(const CharT* d_chars, + const OffsetT block_offset, + const OffsetT num_total_symbols, + cub::Int2Type<false> /*IS_FULL_BLOCK*/, + cub::Int2Type<sizeof(AliasedLoadT)> /*ALIGNMENT*/) + { + AliasedLoadT thread_units[UINTS_PER_THREAD]; + + if (num_total_symbols <= block_offset) return; + + // Last unit to be loaded is IDIV_CEIL(#SYM, SYMBOLS_PER_UNIT) + OffsetT num_total_units = + CUB_QUOTIENT_CEILING(num_total_symbols - block_offset, sizeof(AliasedLoadT)); + + const AliasedLoadT* d_block_symbols = reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); + cub::LoadDirectStriped<BLOCK_THREADS>( + threadIdx.x, d_block_symbols, thread_units, num_total_units); + +#pragma unroll + for (int32_t i = 0; i < UINTS_PER_THREAD; ++i) { + temp_storage.uints[threadIdx.x + i * BLOCK_THREADS] = thread_units[i]; + } + } + + //--------------------------------------------------------------------- + // LOADING BLOCK OF CHARACTERS: DISPATCHER + //--------------------------------------------------------------------- + __device__ __forceinline__ void LoadBlock(const CharT* d_chars, + const OffsetT block_offset, + const OffsetT num_total_symbols) + { + // Check if pointer is aligned to four bytes + if (((uintptr_t)(const void*)(d_chars + block_offset) % 4) == 0) { + if (block_offset + SYMBOLS_PER_UINT_BLOCK < num_total_symbols) { + LoadBlock( + d_chars, block_offset, num_total_symbols, cub::Int2Type<true>(), cub::Int2Type<4>()); + } else { + LoadBlock( + d_chars, block_offset, num_total_symbols, cub::Int2Type<false>(), cub::Int2Type<1>()); + } + } else { + if (block_offset + SYMBOLS_PER_UINT_BLOCK < num_total_symbols) { + LoadBlock( + d_chars, block_offset, num_total_symbols, cub::Int2Type<true>(), cub::Int2Type<1>()); + } else { + LoadBlock( + d_chars, block_offset, num_total_symbols, cub::Int2Type<false>(), cub::Int2Type<1>()); + } + } + } + + template <int32_t NUM_STATES, + typename SymbolMatcherT, + typename TransitionTableT, + typename StateVectorT> + __device__ __forceinline__ void GetThreadStateTransitionVector( + const SymbolMatcherT& symbol_matcher, + const TransitionTableT& transition_table, + const CharT* d_chars, + const OffsetT block_offset, + const OffsetT num_total_symbols, + StateVectorT& state_vector) + { + using StateVectorTransitionOpT = StateVectorTransitionOp<NUM_STATES, StateVectorT, TransitionTableT>; + + // Start parsing and to transition states + StateVectorTransitionOpT transition_op(transition_table, state_vector); + + // Load characters into shared memory + LoadBlock(d_chars, block_offset, num_total_symbols); + + // If this is a full block (i.e., all threads can parse all their symbols) + OffsetT num_block_chars = num_total_symbols - block_offset; + bool is_full_block = (num_block_chars >= SYMBOLS_PER_BLOCK); + + // Ensure characters have been loaded + __syncthreads(); + + // Thread's symbols + CharT* t_chars = &temp_storage.chars[threadIdx.x * SYMBOLS_PER_THREAD]; + + // Parse thread's symbols and transition the state-vector + if (is_full_block) { + GetThreadStateTransitions<SYMBOLS_PER_THREAD>( + symbol_matcher, t_chars, num_block_chars, transition_op, cub::Int2Type<true>()); + } else { + GetThreadStateTransitions<SYMBOLS_PER_THREAD>( + symbol_matcher, t_chars, num_block_chars, transition_op, cub::Int2Type<false>()); + } + + // transition_op.TearDown(); + } + + template <int32_t BYPASS_LOAD, + typename SymbolMatcherT, + typename TransitionTableT, + typename StateVectorT, + typename CallbackOpT> + __device__ __forceinline__ void GetThreadStateTransitions( + SymbolMatcherT const& symbol_matcher, + TransitionTableT const& transition_table, + CharT const* d_chars, + OffsetT const block_offset, + OffsetT const num_total_symbols, + StateVectorT& state_vector, + CallbackOpT& callback_op, + cub::Int2Type<BYPASS_LOAD> /**/) + { + using StateTransitionOpT = StateTransitionOp<CallbackOpT, StateVectorT, TransitionTableT>; + + // Start parsing and to transition states + StateTransitionOpT transition_op(transition_table, state_vector, callback_op); + + // Load characters into shared memory + if (!BYPASS_LOAD) LoadBlock(d_chars, block_offset, num_total_symbols); + + // If this is a full block (i.e., all threads can parse all their symbols) + OffsetT num_block_chars = num_total_symbols - block_offset; + bool is_full_block = (num_block_chars >= SYMBOLS_PER_BLOCK); + + // Ensure characters have been loaded + __syncthreads(); + + // Thread's symbols + CharT* t_chars = &temp_storage.chars[threadIdx.x * SYMBOLS_PER_THREAD]; + + // Initialize callback + callback_op.Init(block_offset + threadIdx.x * SYMBOLS_PER_THREAD); + + // Parse thread's symbols and transition the state-vector + if (is_full_block) { + GetThreadStateTransitions<SYMBOLS_PER_THREAD>( + symbol_matcher, t_chars, num_block_chars, transition_op, cub::Int2Type<true>()); + } else { + GetThreadStateTransitions<SYMBOLS_PER_THREAD>( + symbol_matcher, t_chars, num_block_chars, transition_op, cub::Int2Type<false>()); + } + + callback_op.TearDown(); + } +}; + +template <bool IS_TRANS_VECTOR_PASS, + bool IS_SINGLE_PASS, + typename DfaT, + typename TileStateT, + typename AgentDFAPolicy, + typename SymbolItT, + typename OffsetT, + typename StateVectorT, + typename OutOffsetScanTileState, + typename TransducedOutItT, + typename TransducedIndexOutItT, + typename TransducedCountOutItT> +__launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ + void SimulateDFAKernel(DfaT dfa, + SymbolItT d_chars, + OffsetT const num_chars, + uint32_t seed_state, + StateVectorT* __restrict__ d_thread_state_transition, + TileStateT tile_state, + OutOffsetScanTileState offset_tile_state, + TransducedOutItT transduced_out_it, + TransducedIndexOutItT transduced_out_idx_it, + TransducedCountOutItT d_num_transduced_out_it) +{ + using StateIndexT = uint32_t; + + using AgentDfaSimT = AgentDFA<AgentDFAPolicy, SymbolItT, OffsetT>; + + static constexpr uint32_t NUM_STATES = DfaT::MAX_NUM_STATES; + + enum { + BLOCK_THREADS = AgentDFAPolicy::BLOCK_THREADS, + ITEMS_PER_THREAD = AgentDFAPolicy::ITEMS_PER_THREAD, + SYMBOLS_PER_BLOCK = AgentDfaSimT::SYMBOLS_PER_BLOCK + }; + + // Shared memory required by the DFA simulator + __shared__ typename AgentDfaSimT::TempStorage dfa_storage; + + // Shared memory required by the symbol group lookup table + __shared__ typename DfaT::SymbolGroupStorageT symbol_matcher_storage; + + // Shared memory required by the transition table + __shared__ typename DfaT::TransitionTableStorageT transition_table_storage; + + // Shared memory required by the transducer table + __shared__ typename DfaT::TranslationTableStorageT transducer_table_storage; + + // Initialize symbol group lookup table + auto symbol_matcher = dfa.InitSymbolGroupLUT(symbol_matcher_storage); + + // Initialize transition table + auto transition_table = dfa.InitTransitionTable(transition_table_storage); + + // Initialize transition table + auto transducer_table = dfa.InitTranslationTable(transducer_table_storage); + + // Set up DFA + AgentDfaSimT agent_dfa(dfa_storage); + + // Memory is the state transition vector passed on to the second stage of the algorithm + StateVectorT out_state_vector; + + // Stage 1: Compute the state-transition vector + if (IS_TRANS_VECTOR_PASS || IS_SINGLE_PASS) { + // StateVectorT state_vector; + MultiItemStateVector<int32_t, NUM_STATES> state_vector; + + // Initialize the seed state transition vector with the identity vector +#pragma unroll + for (int32_t i = 0; i < NUM_STATES; ++i) { + state_vector.Set(i, i); + } + + // Compute the state transition vector + agent_dfa.GetThreadStateTransitionVector<NUM_STATES>(symbol_matcher, + transition_table, + d_chars, + blockIdx.x * SYMBOLS_PER_BLOCK, + num_chars, + state_vector); + + // Initialize the state transition vector passed on to the second stage +#pragma unroll + for (int32_t i = 0; i < NUM_STATES; ++i) { + out_state_vector.Set(i, state_vector.Get(i)); + } + + // Write out state-transition vector + if (!IS_SINGLE_PASS) { + d_thread_state_transition[blockIdx.x * BLOCK_THREADS + threadIdx.x] = out_state_vector; + } + } + // Stage 2: Perform FSM simulation + if ((!IS_TRANS_VECTOR_PASS) || IS_SINGLE_PASS) { + constexpr uint32_t SINGLE_ITEM_COUNT = 1; + MultiItemStateVector<int32_t, SINGLE_ITEM_COUNT> state; + + //------------------------------------------------------------------------------ + // SINGLE-PASS: + // -> block-wide inclusive prefix scan on the state transition vector + // -> first block/tile: write out block aggregate as the "tile's" inclusive (i.e., the one that + // incorporates all preceding blocks/tiles results) + //------------------------------------------------------------------------------ + if (IS_SINGLE_PASS) { + uint32_t tile_idx = blockIdx.x; + using StateVectorCompositeOpT = VectorCompositeOp<NUM_STATES>; + + using PrefixCallbackOpT_ = + cub::TilePrefixCallbackOp<StateVectorT, StateVectorCompositeOpT, TileStateT>; + + using ItemsBlockScan = + cub::BlockScan<StateVectorT, BLOCK_THREADS, cub::BlockScanAlgorithm::BLOCK_SCAN_WARP_SCANS>; + + __shared__ typename ItemsBlockScan::TempStorage scan_temp_storage; + __shared__ typename PrefixCallbackOpT_::TempStorage prefix_callback_temp_storage; + + // STATE-TRANSITION IDENTITY VECTOR + StateVectorT state_identity_vector; + for (int32_t i = 0; i < NUM_STATES; ++i) { + state_identity_vector.Set(i, i); + } + StateVectorCompositeOpT state_vector_scan_op; + + // + if (tile_idx == 0) { + StateVectorT block_aggregate; + ItemsBlockScan(scan_temp_storage) + .ExclusiveScan(out_state_vector, + out_state_vector, + state_identity_vector, + state_vector_scan_op, + block_aggregate); + + if (threadIdx.x == 0 /*and not IS_LAST_TILE*/) { + tile_state.SetInclusive(0, block_aggregate); + } + } else { + auto prefix_op = PrefixCallbackOpT_( + tile_state, prefix_callback_temp_storage, state_vector_scan_op, tile_idx); + + ItemsBlockScan(scan_temp_storage) + .ExclusiveScan(out_state_vector, out_state_vector, state_vector_scan_op, prefix_op); + } + __syncthreads(); + state.Set(0, out_state_vector.Get(seed_state)); + } else { + state.Set( + 0, d_thread_state_transition[blockIdx.x * BLOCK_THREADS + threadIdx.x].Get(seed_state)); + } + + // Perform finite-state machine simulation, computing size of transduced output + DFASimulationCallbackWrapper<decltype(dfa.InitTranslationTable(transducer_table_storage)), TransducedOutItT, TransducedIndexOutItT> + callback_wrapper(transducer_table, transduced_out_it, transduced_out_idx_it); + + MultiItemStateVector<int32_t, SINGLE_ITEM_COUNT> t_start_state; + t_start_state.Set(0, state.Get(seed_state)); + agent_dfa.GetThreadStateTransitions(symbol_matcher, + transition_table, + d_chars, + blockIdx.x * SYMBOLS_PER_BLOCK, + num_chars, + state, + callback_wrapper, + cub::Int2Type<IS_SINGLE_PASS>()); + + __syncthreads(); + using OffsetPrefixScanCallbackOpT_ = + cub::TilePrefixCallbackOp<OffsetT, cub::Sum, OutOffsetScanTileState>; + + using OutOffsetBlockScan = + cub::BlockScan<OffsetT, BLOCK_THREADS, cub::BlockScanAlgorithm::BLOCK_SCAN_WARP_SCANS>; + + __shared__ typename OutOffsetBlockScan::TempStorage scan_temp_storage; + __shared__ typename OffsetPrefixScanCallbackOpT_::TempStorage prefix_callback_temp_storage; + + uint32_t tile_idx = blockIdx.x; + if (tile_idx == 0) { + OffsetT block_aggregate = 0; + OutOffsetBlockScan(scan_temp_storage) + .ExclusiveScan(callback_wrapper.out_count, + callback_wrapper.out_count, + static_cast<OffsetT>(0), + cub::Sum{}, + block_aggregate); + + if (threadIdx.x == 0 /*and not IS_LAST_TILE*/) { + offset_tile_state.SetInclusive(0, block_aggregate); + } + + if (tile_idx == gridDim.x - 1 && threadIdx.x == 0) { + *d_num_transduced_out_it = block_aggregate; + } + } else { + auto prefix_op = OffsetPrefixScanCallbackOpT_( + offset_tile_state, prefix_callback_temp_storage, cub::Sum{}, tile_idx); + + OutOffsetBlockScan(scan_temp_storage) + .ExclusiveScan( + callback_wrapper.out_count, callback_wrapper.out_count, cub::Sum{}, prefix_op); + + if (tile_idx == gridDim.x - 1 && threadIdx.x == 0) { + *d_num_transduced_out_it = prefix_op.GetInclusivePrefix(); + } + } + + callback_wrapper.write = true; + agent_dfa.GetThreadStateTransitions(symbol_matcher, + transition_table, + d_chars, + blockIdx.x * SYMBOLS_PER_BLOCK, + num_chars, + t_start_state, + callback_wrapper, + cub::Int2Type<true>()); + } +} + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/device_dfa.cuh b/cpp/src/io/fst/device_dfa.cuh new file mode 100644 index 00000000000..795c4c98bec --- /dev/null +++ b/cpp/src/io/fst/device_dfa.cuh @@ -0,0 +1,264 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 "cub/util_type.cuh" +#include "dispatch_dfa.cuh" +#include <src/io/fst/symbol_lut.cuh> +#include <src/io/fst/transition_table.cuh> +#include <src/io/fst/translation_table.cuh> + +#include <cstdint> + +namespace cudf { +namespace io { +namespace fst { + +/** + * @brief Uses a deterministic finite automaton to transduce a sequence of symbols from an input + * iterator to a sequence of transduced output symbols. + * + * @tparam SymbolItT Random-access input iterator type to symbols fed into the FST + * @tparam DfaT The DFA specification + * @tparam TransducedOutItT Random-access output iterator to which the transduced output will be + * written + * @tparam TransducedIndexOutItT Random-access output iterator type to which the indexes of the + * symbols that caused some output to be written. + * @tparam TransducedCountOutItT A single-item output iterator type to which the total number of + * output symbols is written + * @tparam OffsetT A type large enough to index into either of both: (a) the input symbols and (b) + * the output symbols + * @param[in] d_temp_storage Device-accessible allocation of temporary storage. When NULL, the + * required allocation size is written to \p temp_storage_bytes and no work is done. + * @param[in,out] temp_storage_bytes Reference to size in bytes of \p d_temp_storage allocation + * @param[in] dfa The DFA specifying the number of distinct symbol groups, transition table, and + * translation table + * @param[in] d_chars_in Random-access input iterator to the beginning of the sequence of input + * symbols + * @param[in] num_chars The total number of input symbols to process + * @param[out] transduced_out_it Random-access output iterator to which the transduced output is + * written + * @param[out] transduced_out_idx_it Random-access output iterator to which, the index i is written + * iff the i-th input symbol caused some output to be written + * @param[out] d_num_transduced_out_it A single-item output iterator type to which the total number + * of output symbols is written + * @param[in] seed_state The DFA's starting state. For streaming DFAs this corresponds to the + * "end-state" of the previous invocation of the algorithm. + * @param[in] stream CUDA stream to launch kernels within. Default is the null-stream. + */ +template <typename DfaT, + typename SymbolItT, + typename TransducedOutItT, + typename TransducedIndexOutItT, + typename TransducedCountOutItT, + typename OffsetT> +cudaError_t DeviceTransduce(void* d_temp_storage, + size_t& temp_storage_bytes, + DfaT dfa, + SymbolItT d_chars_in, + OffsetT num_chars, + TransducedOutItT transduced_out_it, + TransducedIndexOutItT transduced_out_idx_it, + TransducedCountOutItT d_num_transduced_out_it, + uint32_t seed_state = 0, + cudaStream_t stream = 0) +{ + using DispatchDfaT = detail::DispatchFSM<DfaT, + SymbolItT, + TransducedOutItT, + TransducedIndexOutItT, + TransducedCountOutItT, + OffsetT>; + + return DispatchDfaT::Dispatch(d_temp_storage, + temp_storage_bytes, + dfa, + seed_state, + d_chars_in, + num_chars, + transduced_out_it, + transduced_out_idx_it, + d_num_transduced_out_it, + stream); +} + +/** + * @brief Helper class to facilitate the specification and instantiation of a DFA (i.e., the + * transition table and its number of states, the mapping of symbols to symbol groups, and the + * translation table that specifies which state transitions cause which output to be written). + * + * @tparam OutSymbolT The symbol type being output by the finite-state transducer + * @tparam NUM_SYMBOLS The number of symbol groups amongst which to differentiate (one dimension of + * the transition table) + * @tparam TT_NUM_STATES The number of states defined by the DFA (the other dimension of the + * transition table) + */ +template <typename OutSymbolT, int32_t NUM_SYMBOLS, int32_t TT_NUM_STATES> +class Dfa { + public: + // The maximum number of states supported by this DFA instance + // This is a value queried by the DFA simulation algorithm + static constexpr int32_t MAX_NUM_STATES = TT_NUM_STATES; + + private: + // Symbol-group id lookup table + using MatcherT = detail::SingleSymbolSmemLUT<char>; + using MatcherInitT = typename MatcherT::KernelParameter; + + // Transition table + using TransitionTableT = detail::TransitionTable<NUM_SYMBOLS + 1, TT_NUM_STATES>; + using TransitionTableInitT = typename TransitionTableT::KernelParameter; + + // Translation lookup table + using OutSymbolOffsetT = uint32_t; + using TransducerTableT = detail::TransducerLookupTable<OutSymbolT, + OutSymbolOffsetT, + NUM_SYMBOLS + 1, + TT_NUM_STATES, + (NUM_SYMBOLS + 1) * TT_NUM_STATES>; + using TransducerTableInitT = typename TransducerTableT::KernelParameter; + + // Private members (passed between host/device) + /// Information to initialize the device-side lookup table that maps symbol -> symbol group id + MatcherInitT symbol_matcher_init; + + /// Information to initialize the device-side transition table + TransitionTableInitT tt_init; + + /// Information to initialize the device-side translation table + TransducerTableInitT tt_out_init; + + public: + //--------------------------------------------------------------------- + // DEVICE-SIDE MEMBER FUNCTIONS + //--------------------------------------------------------------------- + using SymbolGroupStorageT = typename MatcherT::TempStorage; + using TransitionTableStorageT = typename TransitionTableT::TempStorage; + using TranslationTableStorageT = typename TransducerTableT::TempStorage; + + __device__ auto InitSymbolGroupLUT(SymbolGroupStorageT& temp_storage) + { + return MatcherT(symbol_matcher_init, temp_storage); + } + + __device__ auto InitTransitionTable(TransitionTableStorageT& temp_storage) + { + return TransitionTableT(tt_init, temp_storage); + } + + __device__ auto InitTranslationTable(TranslationTableStorageT& temp_storage) + { + return TransducerTableT(tt_out_init, temp_storage); + } + + //--------------------------------------------------------------------- + // HOST-SIDE MEMBER FUNCTIONS + //--------------------------------------------------------------------- + template <typename StateIdT, typename SymbolGroupIdItT> + cudaError_t Init(SymbolGroupIdItT const& symbol_vec, + std::vector<std::vector<StateIdT>> const& tt_vec, + std::vector<std::vector<std::vector<OutSymbolT>>> const& out_tt_vec, + cudaStream_t stream = 0) + { + cudaError_t error = cudaSuccess; + + enum : uint32_t { MEM_SYMBOL_MATCHER = 0, MEM_TT, MEM_OUT_TT, NUM_ALLOCATIONS }; + + size_t allocation_sizes[NUM_ALLOCATIONS] = {0}; + void* allocations[NUM_ALLOCATIONS] = {0}; + + // Memory requirements: lookup table + error = MatcherT::PrepareLUT( + nullptr, allocation_sizes[MEM_SYMBOL_MATCHER], symbol_vec, symbol_matcher_init); + if (error) return error; + + // Memory requirements: transition table + error = + TransitionTableT::CreateTransitionTable(nullptr, allocation_sizes[MEM_TT], tt_vec, tt_init); + if (error) return error; + + // Memory requirements: transducer table + error = TransducerTableT::CreateTransitionTable( + nullptr, allocation_sizes[MEM_OUT_TT], out_tt_vec, tt_out_init); + if (error) return error; + + // Memory requirements: total memory + size_t temp_storage_bytes = 0; + error = cub::AliasTemporaries(nullptr, temp_storage_bytes, allocations, allocation_sizes); + if (error) return error; + + // Allocate memory + void* d_temp_storage = nullptr; + error = cudaMalloc(&d_temp_storage, temp_storage_bytes); + if (error) return error; + + // Alias memory + error = + cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); + if (error) return error; + + // Initialize symbol group lookup table + error = MatcherT::PrepareLUT(allocations[MEM_SYMBOL_MATCHER], + allocation_sizes[MEM_SYMBOL_MATCHER], + symbol_vec, + symbol_matcher_init, + stream); + if (error) return error; + + // Initialize state transition table + error = TransitionTableT::CreateTransitionTable( + allocations[MEM_TT], allocation_sizes[MEM_TT], tt_vec, tt_init, stream); + if (error) return error; + + // Initialize finite-state transducer lookup table + error = TransducerTableT::CreateTransitionTable( + allocations[MEM_OUT_TT], allocation_sizes[MEM_OUT_TT], out_tt_vec, tt_out_init, stream); + if (error) return error; + + return error; + } + + template <typename SymbolT, + typename TransducedOutItT, + typename TransducedIndexOutItT, + typename TransducedCountOutItT, + typename OffsetT> + cudaError_t Transduce(void* d_temp_storage, + size_t& temp_storage_bytes, + SymbolT const* d_chars, + OffsetT num_chars, + TransducedOutItT d_out_it, + TransducedIndexOutItT d_out_idx_it, + TransducedCountOutItT d_num_transduced_out_it, + const uint32_t seed_state = 0, + cudaStream_t stream = 0) + { + return DeviceTransduce(d_temp_storage, + temp_storage_bytes, + *this, + d_chars, + num_chars, + d_out_it, + d_out_idx_it, + d_num_transduced_out_it, + seed_state, + stream); + } +}; + +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/dispatch_dfa.cuh b/cpp/src/io/fst/dispatch_dfa.cuh new file mode 100644 index 00000000000..fc14faaf10a --- /dev/null +++ b/cpp/src/io/fst/dispatch_dfa.cuh @@ -0,0 +1,462 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 "agent_dfa.cuh" +#include "in_reg_array.cuh" + +#include <cub/cub.cuh> + +#include <cstdint> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +/** + * @brief The tuning policy comprising all the architecture-specific compile-time tuning parameters. + * + * @tparam _BLOCK_THREADS Number of threads per block + * @tparam _ITEMS_PER_THREAD Number of symbols processed by each thread + */ +template <int32_t _BLOCK_THREADS, int32_t _ITEMS_PER_THREAD> +struct AgentDFAPolicy { + // The number of threads per block + static constexpr int32_t BLOCK_THREADS = _BLOCK_THREADS; + + // The number of symbols processed by each thread + static constexpr int32_t ITEMS_PER_THREAD = _ITEMS_PER_THREAD; +}; + +/** + * @brief The list of architecture-specific tuning policies. Yet TBD. + */ +struct DeviceFSMPolicy { + //------------------------------------------------------------------------------ + // Architecture-specific tuning policies + //------------------------------------------------------------------------------ + struct Policy900 : cub::ChainedPolicy<900, Policy900, Policy900> { + enum { + BLOCK_THREADS = 128, + ITEMS_PER_THREAD = 32, + }; + + using AgentDFAPolicy = AgentDFAPolicy<BLOCK_THREADS, ITEMS_PER_THREAD>; + }; + + // Top-of-list of the tuning policy "chain" + using MaxPolicy = Policy900; +}; + +/** + * @brief Kernel for initializing single-pass prefix scan tile states + * + * @param items_state The tile state + * @param num_tiles The number of tiles to be initialized + * @return + */ +template <typename TileState> +__global__ void initialization_pass_kernel(TileState items_state, uint32_t num_tiles) +{ + items_state.InitializeStatus(num_tiles); +} + +template <typename DfaT, + typename SymbolItT, + typename TransducedOutItT, + typename TransducedIndexOutItT, + typename TransducedCountOutItT, + typename OffsetT> +struct DispatchFSM : DeviceFSMPolicy { + //------------------------------------------------------------------------------ + // DEFAULT TYPES + //------------------------------------------------------------------------------ + using StateIndexT = uint32_t; + using BlockOffsetT = uint32_t; + + //------------------------------------------------------------------------------ + // DERIVED CONFIGS + //------------------------------------------------------------------------------ + // DFA-specific configs + static constexpr int32_t MAX_NUM_STATES = DfaT::MAX_NUM_STATES; + static constexpr int32_t MAX_NUM_SYMBOLS = DfaT::MAX_NUM_SYMBOLS; + + // Whether to use a single-pass prefix scan that does all in on + static constexpr bool SINGLE_PASS_STV = false; + + // Whether this is a finite-state transform + static constexpr bool IS_FST = true; + + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + using StateVectorCompositeOpT = VectorCompositeOp<MAX_NUM_STATES>; + + //------------------------------------------------------------------------------ + // MEMBER VARS + //------------------------------------------------------------------------------ + void* d_temp_storage; + size_t& temp_storage_bytes; + DfaT dfa; + StateIndexT seed_state; + SymbolItT d_chars_in; + OffsetT num_chars; + TransducedOutItT transduced_out_it; + TransducedIndexOutItT transduced_out_idx_it; + TransducedCountOutItT d_num_transduced_out_it; + cudaStream_t stream; + int ptx_version; + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + CUB_RUNTIME_FUNCTION __forceinline__ DispatchFSM(void* d_temp_storage, + size_t& temp_storage_bytes, + DfaT dfa, + StateIndexT seed_state, + SymbolItT d_chars_in, + OffsetT num_chars, + TransducedOutItT transduced_out_it, + TransducedIndexOutItT transduced_out_idx_it, + TransducedCountOutItT d_num_transduced_out_it, + cudaStream_t stream, + int ptx_version) + : d_temp_storage(d_temp_storage), + temp_storage_bytes(temp_storage_bytes), + dfa(dfa), + seed_state(seed_state), + d_chars_in(d_chars_in), + num_chars(num_chars), + transduced_out_it(transduced_out_it), + transduced_out_idx_it(transduced_out_idx_it), + d_num_transduced_out_it(d_num_transduced_out_it), + stream(stream), + ptx_version(ptx_version) + { + } + + //------------------------------------------------------------------------------ + // DISPATCH INTERFACE + //------------------------------------------------------------------------------ + CUB_RUNTIME_FUNCTION __forceinline__ static cudaError_t Dispatch( + void* d_temp_storage, + size_t& temp_storage_bytes, + DfaT dfa, + StateIndexT seed_state, + SymbolItT d_chars_in, + OffsetT num_chars, + TransducedOutItT transduced_out_it, + TransducedIndexOutItT transduced_out_idx_it, + TransducedCountOutItT d_num_transduced_out_it, + cudaStream_t stream) + { + using MaxPolicyT = DispatchFSM::MaxPolicy; + + cudaError_t error; + + // Get PTX version + int ptx_version; + error = cub::PtxVersion(ptx_version); + if (error) return error; + + // Create dispatch functor + DispatchFSM dispatch(d_temp_storage, + temp_storage_bytes, + dfa, + seed_state, + d_chars_in, + num_chars, + transduced_out_it, + transduced_out_idx_it, + d_num_transduced_out_it, + stream, + ptx_version); + + error = MaxPolicyT::Invoke(ptx_version, dispatch); + return error; + } + + //------------------------------------------------------------------------------ + // DFA SIMULATION KERNEL INVOCATION + //------------------------------------------------------------------------------ + template <typename ActivePolicyT, + typename DFASimulationKernelT, + typename TileStateT, + typename FstScanTileStateT, + typename StateVectorT> + CUB_RUNTIME_FUNCTION __forceinline__ cudaError_t + InvokeDFASimulationKernel(DFASimulationKernelT dfa_kernel, + int32_t sm_count, + StateIndexT seed_state, + StateVectorT* d_thread_state_transition, + TileStateT tile_state, + FstScanTileStateT fst_tile_state) + + { + cudaError_t error = cudaSuccess; + cub::KernelConfig dfa_simulation_config; + + using PolicyT = typename ActivePolicyT::AgentDFAPolicy; + if (CubDebug(error = dfa_simulation_config.Init<PolicyT>(dfa_kernel))) return error; + + // Kernel invocation + uint32_t grid_size = + CUB_QUOTIENT_CEILING(num_chars, PolicyT::BLOCK_THREADS * PolicyT::ITEMS_PER_THREAD); + uint32_t block_threads = dfa_simulation_config.block_threads; + + dfa_kernel<<<grid_size, block_threads, 0, stream>>>(dfa, + d_chars_in, + num_chars, + seed_state, + d_thread_state_transition, + tile_state, + fst_tile_state, + transduced_out_it, + transduced_out_idx_it, + d_num_transduced_out_it); + + // Check for errors + if (CubDebug(error = cudaPeekAtLastError())) return error; + + return error; + } + + /** + * @brief Computes the state-transition vectors + */ + template <typename ActivePolicyT, + typename TileStateT, + typename FstScanTileStateT, + typename StateVectorT> + CUB_RUNTIME_FUNCTION __forceinline__ cudaError_t + ComputeStateTransitionVector(uint32_t sm_count, + TileStateT tile_state, + FstScanTileStateT fst_tile_state, + StateVectorT* d_thread_state_transition) + { + StateIndexT seed_state = 0; + + return InvokeDFASimulationKernel<ActivePolicyT>( + SimulateDFAKernel<true, + SINGLE_PASS_STV, + DfaT, + TileStateT, + typename ActivePolicyT::AgentDFAPolicy, + SymbolItT, + OffsetT, + StateVectorT, + FstScanTileStateT, + TransducedOutItT, + TransducedIndexOutItT, + TransducedCountOutItT>, + sm_count, + seed_state, + d_thread_state_transition, + tile_state, + fst_tile_state); + } + + /** + * @brief Performs the actual DFA simulation. + */ + template <typename ActivePolicyT, + typename TileStateT, + typename FstScanTileStateT, + typename StateVectorT> + CUB_RUNTIME_FUNCTION __forceinline__ cudaError_t + SimulateDFA(uint32_t sm_count, + TileStateT tile_state, + FstScanTileStateT fst_tile_state, + StateIndexT seed_state, + StateVectorT* d_thread_state_transition) + { + return InvokeDFASimulationKernel<ActivePolicyT>( + SimulateDFAKernel<false, + SINGLE_PASS_STV, + DfaT, + TileStateT, + typename ActivePolicyT::AgentDFAPolicy, + SymbolItT, + OffsetT, + StateVectorT, + FstScanTileStateT, + TransducedOutItT, + TransducedIndexOutItT, + TransducedCountOutItT>, + sm_count, + seed_state, + d_thread_state_transition, + tile_state, + fst_tile_state); + } + + //------------------------------------------------------------------------------ + // POLICY INVOKATION + //------------------------------------------------------------------------------ + template <typename ActivePolicyT> + CUB_RUNTIME_FUNCTION __forceinline__ cudaError_t Invoke() + { + cudaError_t error = cudaSuccess; + + // Get SM count + int device_ordinal; + int sm_count; + + // Get current device + error = cudaGetDevice(&device_ordinal); + if (error) + + error = cudaDeviceGetAttribute(&sm_count, cudaDevAttrMultiProcessorCount, device_ordinal); + if (error) return error; + + //------------------------------------------------------------------------------ + // DERIVED TYPEDEFS + //------------------------------------------------------------------------------ + // Type used to represent state-transition vectors + using StateVectorT = MultiFragmentInRegArray<MAX_NUM_STATES, MAX_NUM_STATES - 1>; + + // Scan tile state used for propagating composed state transition vectors + using ScanTileStateT = typename cub::ScanTileState<StateVectorT>; + + // Scan tile state used for propagating transduced output offsets + using FstScanTileStateT = typename cub::ScanTileState<OffsetT>; + + // STATE-TRANSITION IDENTITY VECTOR + StateVectorT state_identity_vector; + for (int32_t i = 0; i < MAX_NUM_STATES; ++i) { + state_identity_vector.Set(i, i); + } + StateVectorCompositeOpT state_vector_scan_op; + + //------------------------------------------------------------------------------ + // DERIVED CONFIGS + //------------------------------------------------------------------------------ + enum { + BLOCK_THREADS = ActivePolicyT::BLOCK_THREADS, + SYMBOLS_PER_THREAD = ActivePolicyT::ITEMS_PER_THREAD, + NUM_SYMBOLS_PER_BLOCK = BLOCK_THREADS * SYMBOLS_PER_THREAD + }; + + BlockOffsetT num_blocks = CUB_QUOTIENT_CEILING(num_chars, NUM_SYMBOLS_PER_BLOCK); + size_t num_threads = num_blocks * BLOCK_THREADS; + + //------------------------------------------------------------------------------ + // TEMPORARY MEMORY REQUIREMENTS + //------------------------------------------------------------------------------ + enum { MEM_STATE_VECTORS = 0, MEM_SCAN, MEM_SINGLE_PASS_STV, MEM_FST_OFFSET, NUM_ALLOCATIONS }; + + size_t allocation_sizes[NUM_ALLOCATIONS] = {0}; + void* allocations[NUM_ALLOCATIONS] = {0}; + + size_t vector_scan_storage_bytes = 0; + + // [MEMORY REQUIREMENTS] STATE-TRANSITION SCAN + cub::DeviceScan::ExclusiveScan(nullptr, + vector_scan_storage_bytes, + static_cast<StateVectorT*>(allocations[MEM_STATE_VECTORS]), + static_cast<StateVectorT*>(allocations[MEM_STATE_VECTORS]), + state_vector_scan_op, + state_identity_vector, + num_threads, + stream); + + allocation_sizes[MEM_STATE_VECTORS] = num_threads * sizeof(StateVectorT); + allocation_sizes[MEM_SCAN] = vector_scan_storage_bytes; + + // Bytes needed for tile status descriptors (fusing state-transition vector + DFA simulation) + if (SINGLE_PASS_STV) { + error = ScanTileStateT::AllocationSize(num_blocks, allocation_sizes[MEM_SINGLE_PASS_STV]); + if (error) return error; + } + + // Bytes needed for tile status descriptors (DFA simulation pass for output size computation + + // output-generating pass) + if (IS_FST) { + error = FstScanTileStateT::AllocationSize(num_blocks, allocation_sizes[MEM_FST_OFFSET]); + if (error) return error; + } + + // Alias the temporary allocations from the single storage blob (or compute the necessary size + // of the blob) + error = + cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); + if (error) return error; + + // Return if the caller is simply requesting the size of the storage allocation + if (d_temp_storage == NULL) return cudaSuccess; + + // Alias memory for state-transition vectors + StateVectorT* d_thread_state_transition = + static_cast<StateVectorT*>(allocations[MEM_STATE_VECTORS]); + + //------------------------------------------------------------------------------ + // INITIALIZE SCAN TILE STATES COMPUTING TRANSDUCED OUTPUT OFFSETS + //------------------------------------------------------------------------------ + FstScanTileStateT fst_offset_tile_state; + if (IS_FST) { + // Construct the tile status (aliases memory internally et al.) + error = fst_offset_tile_state.Init( + num_blocks, allocations[MEM_FST_OFFSET], allocation_sizes[MEM_FST_OFFSET]); + if (error) return error; + constexpr uint32_t FST_INIT_TPB = 256; + uint32_t num_fst_init_blocks = CUB_QUOTIENT_CEILING(num_blocks, FST_INIT_TPB); + initialization_pass_kernel<<<num_fst_init_blocks, FST_INIT_TPB, 0, stream>>>( + fst_offset_tile_state, num_blocks); + } + + //------------------------------------------------------------------------------ + // COMPUTE STATE-TRANSITION VECTORS + //------------------------------------------------------------------------------ + ScanTileStateT stv_tile_state; + if constexpr(SINGLE_PASS_STV) { + // Construct the tile status (aliases memory internally et al.) + error = stv_tile_state.Init( + num_blocks, allocations[MEM_SINGLE_PASS_STV], allocation_sizes[MEM_SINGLE_PASS_STV]); + if (error) return error; + constexpr uint32_t STV_INIT_TPB = 256; + uint32_t num_stv_init_blocks = CUB_QUOTIENT_CEILING(num_blocks, STV_INIT_TPB); + initialization_pass_kernel<<<num_stv_init_blocks, STV_INIT_TPB, 0, stream>>>(stv_tile_state, + num_blocks); + } else { + // Compute state-transition vectors + // TODO tag dispatch or constexpr if depending on single-pass config to avoid superfluous + // template instantiations + ComputeStateTransitionVector<ActivePolicyT>( + sm_count, stv_tile_state, fst_offset_tile_state, d_thread_state_transition); + + // State-transition vector scan computing using the composition operator + cub::DeviceScan::ExclusiveScan(allocations[MEM_SCAN], + allocation_sizes[MEM_SCAN], + d_thread_state_transition, + d_thread_state_transition, + state_vector_scan_op, + state_identity_vector, + num_threads, + stream); + } + + //------------------------------------------------------------------------------ + // SIMULATE DFA + //------------------------------------------------------------------------------ + return SimulateDFA<ActivePolicyT>( + sm_count, stv_tile_state, fst_offset_tile_state, seed_state, d_thread_state_transition); + } +}; +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh new file mode 100644 index 00000000000..f9619c82fe8 --- /dev/null +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -0,0 +1,138 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <cub/cub.cuh> + +#include <cstdint> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +/** + * @brief A bit-packed array of items that can be backed by registers yet allows to be dynamically + * addressed at runtime. The data struture is explained in greater detail in the paper <a + * href="http://www.vldb.org/pvldb/vol13/p616-stehle.pdf">ParPaRaw: Massively Parallel Parsing of + * Delimiter-Separated Raw Data</a>. + * + * @tparam NUM_ITEMS The maximum number of items this data structure is supposed to store + * @tparam MAX_ITEM_VALUE The maximum value that one item can represent + * @tparam BackingFragmentT The data type that is holding the fragments + */ +template <uint32_t NUM_ITEMS, uint32_t MAX_ITEM_VALUE, typename BackingFragmentT = uint32_t> +struct MultiFragmentInRegArray { + /// [b] Minimum number of bits required to represent all values from [0, MAX_ITEM_VALUE] + static constexpr uint32_t MIN_BITS_PER_ITEM = + (MAX_ITEM_VALUE == 0) ? 1 : cub::Log2<(MAX_ITEM_VALUE + 1)>::VALUE; + + /// Number of bits that each fragment can store + static constexpr uint32_t NUM_BITS_PER_FRAGMENT = sizeof(BackingFragmentT) * 8; + + /// [a] The number of bits per fragment per item in the array + static constexpr uint32_t AVAIL_BITS_PER_FRAG_ITEM = NUM_BITS_PER_FRAGMENT / NUM_ITEMS; + + /// [k] The number of bits per item per fragment to be a power of two to avoid costly integer + /// multiplication + /// TODO: specialise for VOLTA and later architectures that have efficient integer multiplication + static constexpr uint32_t BITS_PER_FRAG_ITEM = + 0x01U << (cub::Log2<(AVAIL_BITS_PER_FRAG_ITEM + 1)>::VALUE - 1); + static constexpr uint32_t LOG2_BITS_PER_FRAG_ITEM = cub::Log2<BITS_PER_FRAG_ITEM>::VALUE; + + // [f] Number of fragments required to store and to reconstruct an item + static constexpr uint32_t FRAGMENTS_PER_ITEM = + (MIN_BITS_PER_ITEM + BITS_PER_FRAG_ITEM - 1) / BITS_PER_FRAG_ITEM; + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + __device__ __host__ __forceinline__ unsigned int bfe(const unsigned int& data, + unsigned int bit_start, + unsigned int num_bits) const + { +#if CUB_PTX_ARCH > 0 + return cub::BFE(data, bit_start, num_bits); +#else + const unsigned int MASK = (1 << num_bits) - 1; + return (data >> bit_start) & MASK; +#endif + } + + __device__ __host__ __forceinline__ void bfi(unsigned int& data, + unsigned int bits, + unsigned int bit_start, + unsigned int num_bits) const + { +#if CUB_PTX_ARCH > 0 + cub::BFI(data, data, bits, bit_start, num_bits); +#else + unsigned int x = bits << bit_start; + unsigned int y = data; + unsigned int MASK_X = ((1 << num_bits) - 1) << bit_start; + unsigned int MASK_Y = ~MASK_X; + data = (y & MASK_Y) | (x & MASK_X); +#endif + } + + BackingFragmentT data[FRAGMENTS_PER_ITEM]; + + //------------------------------------------------------------------------------ + // ACCESSORS + //------------------------------------------------------------------------------ + __host__ __device__ __forceinline__ uint32_t Get(int32_t index) const + { + uint32_t val = 0; + + // #pragma unroll + for (uint32_t i = 0; i < FRAGMENTS_PER_ITEM; ++i) { + val = val | bfe(data[i], index * BITS_PER_FRAG_ITEM, BITS_PER_FRAG_ITEM) + << (i * BITS_PER_FRAG_ITEM); + } + return val; + } + + __host__ __device__ __forceinline__ void Set(uint32_t index, uint32_t value) + { + // #pragma unroll + for (uint32_t i = 0; i < FRAGMENTS_PER_ITEM; ++i) { + uint32_t frag_bits = bfe(value, i * BITS_PER_FRAG_ITEM, BITS_PER_FRAG_ITEM); + bfi(data[i], frag_bits, index * BITS_PER_FRAG_ITEM, BITS_PER_FRAG_ITEM); + } + } + + //------------------------------------------------------------------------------ + // CONSTRUCTORS + //------------------------------------------------------------------------------ + __host__ __device__ __forceinline__ MultiFragmentInRegArray() + { + for (uint32_t i = 0; i < FRAGMENTS_PER_ITEM; ++i) { + data[i] = 0; + } + } + + __host__ __device__ __forceinline__ MultiFragmentInRegArray(uint32_t const (&array)[NUM_ITEMS]) + { + for (uint32_t i = 0; i < NUM_ITEMS; ++i) { + Set(i, array[i]); + } + } +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/symbol_lut.cuh b/cpp/src/io/fst/symbol_lut.cuh new file mode 100644 index 00000000000..08d5f4db58d --- /dev/null +++ b/cpp/src/io/fst/symbol_lut.cuh @@ -0,0 +1,182 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <cub/cub.cuh> + +#include <algorithm> +#include <cstdint> +#include <vector> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { +/** + * @brief Class template that can be plugged into the finite-state machine to look up the symbol + * group index for a given symbol. Class template does not support multi-symbol lookups (i.e., no + * look-ahead). + * + * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id + */ +template <typename SymbolT> +struct SingleSymbolSmemLUT { + //------------------------------------------------------------------------------ + // DEFAULT TYPEDEFS + //------------------------------------------------------------------------------ + // Type used for representing a symbol group id (i.e., what we return for a given symbol) + using SymbolGroupIdT = uint8_t; + + //------------------------------------------------------------------------------ + // DERIVED CONFIGURATIONS + //------------------------------------------------------------------------------ + /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) + static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); + + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + + struct _TempStorage { + // d_match_meta_data[symbol] -> symbol group index + SymbolGroupIdT match_meta_data[NUM_ENTRIES_PER_LUT]; + }; + + struct KernelParameter { + // d_match_meta_data[min(symbol,num_valid_entries)] -> symbol group index + SymbolGroupIdT num_valid_entries; + + // d_match_meta_data[symbol] -> symbol group index + SymbolGroupIdT* d_match_meta_data; + }; + + struct TempStorage : cub::Uninitialized<_TempStorage> { + }; + + //------------------------------------------------------------------------------ + // HELPER METHODS + //------------------------------------------------------------------------------ + /** + * @brief + * + * @param[in] d_temp_storage Device-side temporary storage that can be used to store the lookup + * table. If no storage is provided it will return the temporary storage requirements in \p + * d_temp_storage_bytes. + * @param[in,out] d_temp_storage_bytes Amount of device-side temporary storage that can be used in + * the number of bytes + * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols + * (characters!) that correspond to the i-th symbol group index + * @param[out] kernel_param The kernel parameter object to be initialized with the given mapping + * of symbols to symbol group ids. + * @param[in] stream The stream that shall be used to cudaMemcpyAsync the lookup table + * @return + */ + template <typename SymbolGroupItT> + __host__ __forceinline__ static cudaError_t PrepareLUT(void* d_temp_storage, + size_t& d_temp_storage_bytes, + SymbolGroupItT const& symbol_strings, + KernelParameter& kernel_param, + cudaStream_t stream = 0) + { + // The symbol group index to be returned if none of the given symbols match + SymbolGroupIdT no_match_id = symbol_strings.size(); + + std::vector<SymbolGroupIdT> lut(NUM_ENTRIES_PER_LUT); + SymbolGroupIdT max_base_match_val = 0; + + // Initialize all entries: by default we return the no-match-id + for (uint32_t i = 0; i < NUM_ENTRIES_PER_LUT; ++i) { + lut[i] = no_match_id; + } + + // Set up lookup table + uint32_t sg_id = 0; + for (auto const& sg_symbols : symbol_strings) { + for (auto const& sg_symbol : sg_symbols) { + max_base_match_val = std::max(max_base_match_val, static_cast<SymbolGroupIdT>(sg_symbol)); + lut[sg_symbol] = sg_id; + } + sg_id++; + } + + // Initialize the out-of-bounds lookup: d_match_meta_data[max_base_match_val+1] -> no_match_id + lut[max_base_match_val + 1] = no_match_id; + + // Alias memory / return memory requiremenets + kernel_param.num_valid_entries = max_base_match_val + 2; + if (d_temp_storage) { + cudaError_t error = cudaMemcpyAsync(d_temp_storage, + lut.data(), + kernel_param.num_valid_entries * sizeof(SymbolGroupIdT), + cudaMemcpyHostToDevice, + stream); + + kernel_param.d_match_meta_data = reinterpret_cast<SymbolGroupIdT*>(d_temp_storage); + return error; + } else { + d_temp_storage_bytes = kernel_param.num_valid_entries * sizeof(SymbolGroupIdT); + return cudaSuccess; + } + + return cudaSuccess; + } + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + SymbolGroupIdT num_valid_entries; + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + __host__ __device__ __forceinline__ SingleSymbolSmemLUT(KernelParameter const& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) + { + // GPU-side init +#if CUB_PTX_ARCH > 0 + for (int32_t i = threadIdx.x; i < kernel_param.num_valid_entries; i += blockDim.x) { + this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + } + __syncthreads(); + +#else + // CPU-side init + for (std::size_t i = 0; i < kernel_param.num_luts; i++) { + this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + } +#endif + } + + __host__ __device__ __forceinline__ int32_t operator()(SymbolT const symbol) const + { + // Look up the symbol group for given symbol + return temp_storage.match_meta_data[min(symbol, num_valid_entries - 1)]; + } +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/transition_table.cuh b/cpp/src/io/fst/transition_table.cuh new file mode 100644 index 00000000000..97fef03d8af --- /dev/null +++ b/cpp/src/io/fst/transition_table.cuh @@ -0,0 +1,149 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <cub/cub.cuh> + +#include <cstdint> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +template <int MAX_NUM_SYMBOLS, int MAX_NUM_STATES> +struct TransitionTable { + //------------------------------------------------------------------------------ + // DEFAULT TYPEDEFS + //------------------------------------------------------------------------------ + using ItemT = char; + + struct TransitionVectorWrapper { + const ItemT* data; + + __host__ __device__ TransitionVectorWrapper(const ItemT* data) : data(data) {} + + __host__ __device__ __forceinline__ uint32_t Get(int32_t index) const { return data[index]; } + }; + + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + using TransitionVectorT = TransitionVectorWrapper; + + struct _TempStorage { + // + ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; + }; + + struct TempStorage : cub::Uninitialized<_TempStorage> { + }; + + struct KernelParameter { + ItemT* transitions; + }; + + using LoadAliasT = std::uint32_t; + + static constexpr std::size_t NUM_AUX_MEM_BYTES = + CUB_QUOTIENT_CEILING(MAX_NUM_STATES * MAX_NUM_SYMBOLS * sizeof(ItemT), sizeof(LoadAliasT)) * + sizeof(LoadAliasT); + + //------------------------------------------------------------------------------ + // HELPER METHODS + //------------------------------------------------------------------------------ + __host__ static cudaError_t CreateTransitionTable( + void* d_temp_storage, + size_t& temp_storage_bytes, + const std::vector<std::vector<int>>& trans_table, + KernelParameter& kernel_param, + cudaStream_t stream = 0) + { + if (!d_temp_storage) { + temp_storage_bytes = NUM_AUX_MEM_BYTES; + return cudaSuccess; + } + + // trans_vectors[symbol][state] -> new_state + ItemT trans_vectors[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; + + // trans_table[state][symbol] -> new state + for (std::size_t state = 0; state < trans_table.size(); ++state) { + for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { + trans_vectors[symbol * MAX_NUM_STATES + state] = trans_table[state][symbol]; + } + } + + kernel_param.transitions = static_cast<ItemT*>(d_temp_storage); + + // Copy transition table to device + return cudaMemcpyAsync( + d_temp_storage, trans_vectors, NUM_AUX_MEM_BYTES, cudaMemcpyHostToDevice, stream); + } + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __host__ __device__ __forceinline__ TransitionTable(const KernelParameter& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()) + { +#if CUB_PTX_ARCH > 0 + for (int i = threadIdx.x; i < CUB_QUOTIENT_CEILING(NUM_AUX_MEM_BYTES, sizeof(LoadAliasT)); + i += blockDim.x) { + reinterpret_cast<LoadAliasT*>(this->temp_storage.transitions)[i] = + reinterpret_cast<LoadAliasT*>(kernel_param.transitions)[i]; + } + __syncthreads(); +#else + for (int i = 0; i < kernel_param.num_luts; i++) { + this->temp_storage.transitions[i] = kernel_param.transitions[i]; + } +#endif + } + + /** + * @brief Returns a random-access iterator to lookup all the state transitions for one specific + * symbol from an arbitrary old_state, i.e., it[old_state] -> new_state. + * + * @param state_id The DFA's current state index from which we'll transition + * @param match_id The symbol group id of the symbol that we just read in + * @return + */ + template <typename StateIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ int32_t operator()(StateIndexT state_id, + SymbolIndexT match_id) const + { + return temp_storage.transitions[match_id * MAX_NUM_STATES + state_id]; + } +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/translation_table.cuh b/cpp/src/io/fst/translation_table.cuh new file mode 100644 index 00000000000..bfbfd41e3f0 --- /dev/null +++ b/cpp/src/io/fst/translation_table.cuh @@ -0,0 +1,200 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 "in_reg_array.cuh" + +#include <cub/cub.cuh> + +#include <cstdint> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +/** + * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols to + * output + * + * @tparam OutSymbolT The symbol type being returned + * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols + * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition + * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support + * @tparam MAX_TABLE_SIZE The maximum number of items in the lookup table of output symbols + */ +template <typename OutSymbolT, + typename OutSymbolOffsetT, + int32_t MAX_NUM_SYMBOLS, + int32_t MAX_NUM_STATES, + int32_t MAX_TABLE_SIZE = (MAX_NUM_SYMBOLS * MAX_NUM_STATES)> +struct TransducerLookupTable { + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + struct _TempStorage { + OutSymbolOffsetT out_offset[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; + OutSymbolT out_symbols[MAX_TABLE_SIZE]; + }; + + struct TempStorage : cub::Uninitialized<_TempStorage> { + }; + + struct KernelParameter { + OutSymbolOffsetT* d_trans_offsets; + OutSymbolT* d_out_symbols; + }; + + //------------------------------------------------------------------------------ + // HELPER METHODS + //------------------------------------------------------------------------------ + __host__ static cudaError_t CreateTransitionTable( + void* d_temp_storage, + size_t& temp_storage_bytes, + const std::vector<std::vector<std::vector<OutSymbolT>>>& trans_table, + KernelParameter& kernel_param, + cudaStream_t stream = 0) + { + enum { MEM_OFFSETS = 0, MEM_OUT_SYMBOLS, NUM_ALLOCATIONS }; + + size_t allocation_sizes[NUM_ALLOCATIONS] = {}; + void* allocations[NUM_ALLOCATIONS] = {}; + allocation_sizes[MEM_OFFSETS] = + (MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1) * sizeof(OutSymbolOffsetT); + allocation_sizes[MEM_OUT_SYMBOLS] = MAX_TABLE_SIZE * sizeof(OutSymbolT); + + // Alias the temporary allocations from the single storage blob (or compute the necessary size + // of the blob) + cudaError_t error = + cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); + if (error) return error; + + // Return if the caller is simply requesting the size of the storage allocation + if (d_temp_storage == nullptr) return cudaSuccess; + + std::vector<OutSymbolT> out_symbols; + out_symbols.reserve(MAX_TABLE_SIZE); + std::vector<OutSymbolOffsetT> out_symbol_offsets; + out_symbol_offsets.reserve(MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1); + out_symbol_offsets.push_back(0); + + int st = 0; + // Iterate over the states in the transition table + for (auto const& state_trans : trans_table) { + uint32_t num_added = 0; + // Iterate over the symbols in the transition table + for (auto const& symbol_out : state_trans) { + // Insert the output symbols for this specific (state, symbol) transition + out_symbols.insert(std::end(out_symbols), std::begin(symbol_out), std::end(symbol_out)); + out_symbol_offsets.push_back(out_symbols.size()); + num_added++; + } + st++; + + // Copy the last offset for all symbols (to guarantee a proper lookup for omitted symbols of + // this state) + if (MAX_NUM_SYMBOLS > num_added) { + int32_t count = MAX_NUM_SYMBOLS - num_added; + auto begin_it = std::prev(std::end(out_symbol_offsets)); + std::copy(begin_it, begin_it + count, std::back_inserter(out_symbol_offsets)); + } + } + + // Check whether runtime-provided table size exceeds the compile-time given max. table size + if (out_symbols.size() > MAX_TABLE_SIZE) { return cudaErrorInvalidValue; } + + kernel_param.d_trans_offsets = static_cast<OutSymbolOffsetT*>(allocations[MEM_OFFSETS]); + kernel_param.d_out_symbols = static_cast<OutSymbolT*>(allocations[MEM_OUT_SYMBOLS]); + + // Copy out symbols + error = cudaMemcpyAsync(kernel_param.d_trans_offsets, + out_symbol_offsets.data(), + out_symbol_offsets.size() * sizeof(out_symbol_offsets[0]), + cudaMemcpyHostToDevice, + stream); + if (error) { return error; } + + // Copy offsets into output symbols + return cudaMemcpyAsync(kernel_param.d_out_symbols, + out_symbols.data(), + out_symbols.size() * sizeof(out_symbols[0]), + cudaMemcpyHostToDevice, + stream); + } + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __host__ __device__ __forceinline__ TransducerLookupTable(const KernelParameter& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()) + { + constexpr uint32_t num_offsets = MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1; +#if CUB_PTX_ARCH > 0 + for (int i = threadIdx.x; i < num_offsets; i += blockDim.x) { + this->temp_storage.out_offset[i] = kernel_param.d_trans_offsets[i]; + } + // Make sure all threads in the block can read out_symbol_offsets[num_offsets - 1] from shared + // memory + __syncthreads(); + for (int i = threadIdx.x; i < this->temp_storage.out_offset[num_offsets - 1]; i += blockDim.x) { + this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; + } + __syncthreads(); +#else + for (int i = 0; i < num_offsets; i++) { + this->temp_storage.out_symbol_offsets[i] = kernel_param.d_trans_offsets[i]; + } + for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { + this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; + } +#endif + } + + template <typename StateIndexT, typename SymbolIndexT, typename RelativeOffsetT> + __host__ __device__ __forceinline__ OutSymbolT operator()(StateIndexT state_id, + SymbolIndexT match_id, + RelativeOffsetT relative_offset) const + { + auto offset = temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id] + relative_offset; + return temp_storage.out_symbols[offset]; + } + + template <typename StateIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ OutSymbolOffsetT operator()(StateIndexT state_id, + SymbolIndexT match_id) const + { + return temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id + 1] - + temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id]; + } +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/tests/CMakeLists.txt b/cpp/tests/CMakeLists.txt index a1e3cfed286..587f317692f 100644 --- a/cpp/tests/CMakeLists.txt +++ b/cpp/tests/CMakeLists.txt @@ -225,6 +225,7 @@ ConfigureTest(JSON_TEST io/json_test.cpp) ConfigureTest(ARROW_IO_SOURCE_TEST io/arrow_io_source_test.cpp) ConfigureTest(MULTIBYTE_SPLIT_TEST io/text/multibyte_split_test.cpp) ConfigureTest(LOGICAL_STACK_TEST io/fst/logical_stack_test.cu) +ConfigureTest(FST_TEST io/fst/fst_test.cu) if(CUDF_ENABLE_ARROW_S3) target_compile_definitions(ARROW_IO_SOURCE_TEST PRIVATE "S3_ENABLED") endif() diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu new file mode 100644 index 00000000000..26bb9d47dca --- /dev/null +++ b/cpp/tests/io/fst/fst_test.cu @@ -0,0 +1,291 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <io/fst/device_dfa.cuh> +#include <io/utilities/hostdevice_vector.hpp> + +#include <cudf_test/base_fixture.hpp> +#include <cudf_test/cudf_gtest.hpp> + +#include <cudf/types.hpp> + +#include <rmm/cuda_stream_view.hpp> +#include <rmm/device_buffer.hpp> +#include <rmm/device_uvector.hpp> + +#include "cub/cub.cuh" + +#include <cstdlib> +#include <vector> + +namespace { + +//------------------------------------------------------------------------------ +// CPU-BASED IMPLEMENTATIONS FOR VERIFICATION +//------------------------------------------------------------------------------ +/** + * @brief CPU-based implementation of a finite-state transducer (FST). + * + * @tparam InputItT Forward input iterator type to symbols fed into the FST + * @tparam StateT Type representing states of the finite-state machine + * @tparam SymbolGroupLutT Sequence container of symbol groups. Each symbol group is a sequence + * container to symbols within that group. + * @tparam TransitionTableT Two-dimensional container type + * @tparam TransducerTableT Two-dimensional container type + * @tparam OutputItT Forward output iterator type + * @tparam IndexOutputItT Forward output iterator type + * @param[in] begin Forward iterator to the beginning of the symbol sequence + * @param[in] end Forward iterator to one past the last element of the symbol sequence + * @param[in] init_state The starting state of the finite-state machine + * @param[in] symbol_group_lut Sequence container of symbol groups. Each symbol group is a sequence + * container to symbols within that group. The index of the symbol group containing a symbol being + * read will be used as symbol_gid of the transition and translation tables. + * @param[in] transition_table The two-dimensional transition table, i.e., + * transition_table[state][symbol_gid] -> new_state + * @param[in] translation_table The two-dimensional transducer table, i.e., + * translation_table[state][symbol_gid] -> range_of_output_symbols + * @param[out] out_tape A forward output iterator to which the transduced input will be written + * @param[out] out_index_tape A forward output iterator to which indexes of the symbols that + * actually caused some output are written to + * @return A pair of iterators to one past the last element of (1) the transduced output symbol + * sequence and (2) the indexes of + */ +template <typename InputItT, + typename StateT, + typename SymbolGroupLutT, + typename TransitionTableT, + typename TransducerTableT, + typename OutputItT, + typename IndexOutputItT> +static std::pair<OutputItT, IndexOutputItT> fst_baseline(InputItT begin, + InputItT end, + StateT const& init_state, + SymbolGroupLutT symbol_group_lut, + TransitionTableT transition_table, + TransducerTableT translation_table, + OutputItT out_tape, + IndexOutputItT out_index_tape) +{ + // Initialize "FSM" with starting state + StateT state = init_state; + + // To track the symbol offset within the input that caused the FST to output + std::size_t in_offset = 0; + for (auto it = begin; it < end; it++) { + // The symbol currently being read + auto const& symbol = *it; + + std::size_t symbol_group = 0; + bool found = false; + + // Iterate over symbol groups and search for the first symbol group containing the current + // symbol + for (auto const& sg : symbol_group_lut) { + for (auto const& s : sg) + if (s == symbol) found = true; + if (found) break; + symbol_group++; + } + + // Output the translated symbols to the output tape + size_t inserted = 0; + for (auto out : translation_table[state][symbol_group]) { + // std::cout << in_offset << ": " << out << "\n"; + *out_tape = out; + ++out_tape; + inserted++; + } + + // Output the index of the current symbol, iff it caused some output to be written + if (inserted > 0) { + *out_index_tape = in_offset; + out_index_tape++; + } + + // Transition the state of the finite-state machine + state = transition_table[state][symbol_group]; + + in_offset++; + } + return {out_tape, out_index_tape}; +} + +//------------------------------------------------------------------------------ +// TEST FST SPECIFICATIONS +//------------------------------------------------------------------------------ +// FST to check for brackets and braces outside of pairs of quotes +// The state being active while being outside of a string. When encountering an opening bracket +// or curly brace, we push it onto the stack. When encountering a closing bracket or brace, we +// pop it from the stack. +constexpr uint32_t TT_OOS = 0U; + +// The state being active while being within a string (e.g., field name or a string value). We do +// not push or pop from the stack while being in this state. +constexpr uint32_t TT_STR = 1U; + +// The state being active after encountering an escape symbol (e.g., '\') while being in the TT_STR +// state. constexpr uint32_t TT_ESC = 2U; // cmt to avoid 'unused' warning + +// Total number of states +constexpr uint32_t TT_NUM_STATES = 3U; + +// Definition of the symbol groups +enum PDA_SG_ID { + OBC = 0U, ///< Opening brace SG: { + OBT, ///< Opening bracket SG: [ + CBC, ///< Closing brace SG: } + CBT, ///< Closing bracket SG: ] + QTE, ///< Quote character SG: " + ESC, ///< Escape character SG: '\' + OTR, ///< SG implicitly matching all other characters + NUM_SYMBOL_GROUPS ///< Total number of symbol groups +}; + +// Transition table +const std::vector<std::vector<int32_t>> pda_state_tt = { + /* IN_STATE { [ } ] " \ OTHER */ + /* TT_OOS */ {TT_OOS, TT_OOS, TT_OOS, TT_OOS, TT_STR, TT_OOS, TT_OOS}, + /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_STR, TT_STR}, + /* TT_ESC */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR}}; + +// Translation table (i.e., for each transition, what are the symbols that we output) +const std::vector<std::vector<std::vector<char>>> pda_out_tt = { + /* IN_STATE { [ } ] " \ OTHER */ + /* TT_OOS */ {{'{'}, {'['}, {'}'}, {']'}, {'x'}, {'x'}, {'x'}}, + /* TT_STR */ {{'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}}, + /* TT_ESC */ {{'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}}}; + +// The i-th string representing all the characters of a symbol group +const std::vector<std::string> pda_sgs = {"{", "[", "}", "]", "\"", "\\"}; + +// The DFA's starting state +constexpr int32_t start_state = TT_OOS; + +} // namespace + +// Base test fixture for tests +struct FstTest : public cudf::test::BaseFixture { +}; + +TEST_F(FstTest, GroundTruth) +{ + // Type used to represent the atomic symbol type used within the finite-state machine + using SymbolT = char; + + // Type sufficiently large to index symbols within the input and output (may be unsigned) + using SymbolOffsetT = uint32_t; + + // Helper class to set up transition table, symbol group lookup table, and translation table + using DfaFstT = cudf::io::fst::Dfa<char, (NUM_SYMBOL_GROUPS - 1), TT_NUM_STATES>; + + // Prepare cuda stream for data transfers & kernels + cudaStream_t stream = nullptr; + cudaStreamCreate(&stream); + rmm::cuda_stream_view stream_view(stream); + + // Test input + std::string input = R"( {)" + R"(category": "reference",)" + R"("index:" [4,12,42],)" + R"("author": "Nigel Rees",)" + R"("title": "Sayings of the Century",)" + R"("price": 8.95)" + R"(} )" + R"({)" + R"("category": "reference",)" + R"("index:" [4,{},null,{"a":[]}],)" + R"("author": "Nigel Rees",)" + R"("title": "Sayings of the Century",)" + R"("price": 8.95)" + R"(} {} [] [ ])"; + + // Repeat input sample 1024x + for (std::size_t i = 0; i < 10; i++) + input += input; + + // Prepare input & output buffers + rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); + hostdevice_vector<SymbolT> output_gpu(input.size(), stream_view); + hostdevice_vector<SymbolOffsetT> out_indexes_gpu(input.size(), stream_view); + ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( + d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); + + // Run algorithm + DfaFstT parser; + + // Initialize DFA + ASSERT_CUDA_SUCCEEDED(parser.Init(pda_sgs, pda_state_tt, pda_out_tt, stream)); + + std::size_t temp_storage_bytes = 0; + + // Query temporary storage requirements + ASSERT_CUDA_SUCCEEDED(parser.Transduce(nullptr, + temp_storage_bytes, + d_input.data(), + static_cast<SymbolOffsetT>(d_input.size()), + output_gpu.device_ptr(), + out_indexes_gpu.device_ptr(), + cub::DiscardOutputIterator<int32_t>{}, + start_state, + stream)); + + // Allocate device-side temporary storage & run algorithm + rmm::device_buffer temp_storage{temp_storage_bytes, stream_view}; + ASSERT_CUDA_SUCCEEDED(parser.Transduce(temp_storage.data(), + temp_storage_bytes, + d_input.data(), + static_cast<SymbolOffsetT>(d_input.size()), + output_gpu.device_ptr(), + out_indexes_gpu.device_ptr(), + cub::DiscardOutputIterator<int32_t>{}, + start_state, + stream)); + + // Async copy results from device to host + output_gpu.device_to_host(stream_view); + out_indexes_gpu.device_to_host(stream_view); + + // Prepare CPU-side results for verification + std::string output_cpu{}; + std::vector<SymbolOffsetT> out_index_cpu{}; + output_cpu.reserve(input.size()); + out_index_cpu.reserve(input.size()); + + // Run CPU-side algorithm + fst_baseline(std::begin(input), + std::end(input), + start_state, + pda_sgs, + pda_state_tt, + pda_out_tt, + std::back_inserter(output_cpu), + std::back_inserter(out_index_cpu)); + + // Make sure results have been copied back to host + cudaStreamSynchronize(stream); + + // Verify results + ASSERT_EQ(output_gpu.size(), output_cpu.size()); + ASSERT_EQ(out_indexes_gpu.size(), out_index_cpu.size()); + for (std::size_t i = 0; i < output_gpu.size(); i++) { + ASSERT_EQ(output_gpu.host_ptr()[i], output_cpu[i]) << "Mismatch at index #" << i; + } + for (std::size_t i = 0; i < out_indexes_gpu.size(); i++) { + ASSERT_EQ(out_indexes_gpu.host_ptr()[i], out_index_cpu[i]) << "Mismatch at index #" << i; + } +} + +CUDF_TEST_PROGRAM_MAIN() From 355d1e43e29e4eeadc21f9d4d9e6aa43ee8afe9b Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Apr 2022 05:11:32 -0700 Subject: [PATCH 02/81] clean up & addressing review comments --- cpp/src/io/fst/agent_dfa.cuh | 28 ++++++++++++---------- cpp/src/io/fst/dispatch_dfa.cuh | 4 ++-- cpp/src/io/fst/in_reg_array.cuh | 42 ++++++++++++++++----------------- 3 files changed, 38 insertions(+), 36 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index d983f9287a9..0611973f78c 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -17,8 +17,6 @@ #include "in_reg_array.cuh" -#include <cudf/types.hpp> - #include <cub/cub.cuh> namespace cudf { @@ -40,13 +38,13 @@ template <typename StateIndexT, int32_t NUM_ITEMS> class MultiItemStateVector { public: template <typename IndexT> - constexpr CUDF_HOST_DEVICE void Set(IndexT index, StateIndexT value) noexcept + __host__ __device__ __forceinline__ void Set(IndexT index, StateIndexT value) noexcept { state_[index] = value; } template <typename IndexT> - constexpr CUDF_HOST_DEVICE StateIndexT Get(IndexT index) const noexcept + __host__ __device__ __forceinline__ StateIndexT Get(IndexT index) const noexcept { return state_[index]; } @@ -71,7 +69,7 @@ class MultiItemStateVector { template <int32_t NUM_ITEMS> struct VectorCompositeOp { template <typename VectorT> - constexpr CUDF_HOST_DEVICE VectorT operator()(VectorT const& lhs, VectorT const& rhs) + __host__ __device__ __forceinline__ VectorT operator()(VectorT const& lhs, VectorT const& rhs) { VectorT res; for (int32_t i = 0; i < NUM_ITEMS; ++i) { @@ -109,8 +107,7 @@ class DFASimulationCallbackWrapper { uint32_t count = transducer_table(old_state.Get(0), symbol_id); if (write) { for (uint32_t out_char = 0; out_char < count; out_char++) { - out_it[out_count + out_char] = - transducer_table(old_state.Get(0), symbol_id, out_char); + out_it[out_count + out_char] = transducer_table(old_state.Get(0), symbol_id, out_char); out_idx_it[out_count + out_char] = offset + character_index; } } @@ -188,8 +185,8 @@ struct StateTransitionOp { __host__ __device__ __forceinline__ void ReadSymbol(const CharIndexT& character_index, const SymbolIndexT& read_symbol_id) { - using TransitionVectorT= typename TransitionTableT::TransitionVectorT ; - old_state_vector = state_vector; + using TransitionVectorT = typename TransitionTableT::TransitionVectorT; + old_state_vector = state_vector; state_vector.Set(0, transition_table(state_vector.Get(0), read_symbol_id)); callback_op.ReadSymbol(character_index, old_state_vector, state_vector, read_symbol_id); } @@ -344,7 +341,8 @@ struct AgentDFA { { AliasedLoadT thread_units[UINTS_PER_THREAD]; - const AliasedLoadT* d_block_symbols = reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); + const AliasedLoadT* d_block_symbols = + reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); cub::LoadDirectStriped<BLOCK_THREADS>(threadIdx.x, d_block_symbols, thread_units); #pragma unroll @@ -370,7 +368,8 @@ struct AgentDFA { OffsetT num_total_units = CUB_QUOTIENT_CEILING(num_total_symbols - block_offset, sizeof(AliasedLoadT)); - const AliasedLoadT* d_block_symbols = reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); + const AliasedLoadT* d_block_symbols = + reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); cub::LoadDirectStriped<BLOCK_THREADS>( threadIdx.x, d_block_symbols, thread_units, num_total_units); @@ -419,7 +418,8 @@ struct AgentDFA { const OffsetT num_total_symbols, StateVectorT& state_vector) { - using StateVectorTransitionOpT = StateVectorTransitionOp<NUM_STATES, StateVectorT, TransitionTableT>; + using StateVectorTransitionOpT = + StateVectorTransitionOp<NUM_STATES, StateVectorT, TransitionTableT>; // Start parsing and to transition states StateVectorTransitionOpT transition_op(transition_table, state_vector); @@ -650,7 +650,9 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ } // Perform finite-state machine simulation, computing size of transduced output - DFASimulationCallbackWrapper<decltype(dfa.InitTranslationTable(transducer_table_storage)), TransducedOutItT, TransducedIndexOutItT> + DFASimulationCallbackWrapper<decltype(dfa.InitTranslationTable(transducer_table_storage)), + TransducedOutItT, + TransducedIndexOutItT> callback_wrapper(transducer_table, transduced_out_it, transduced_out_idx_it); MultiItemStateVector<int32_t, SINGLE_ITEM_COUNT> t_start_state; diff --git a/cpp/src/io/fst/dispatch_dfa.cuh b/cpp/src/io/fst/dispatch_dfa.cuh index fc14faaf10a..316d6ea0d5f 100644 --- a/cpp/src/io/fst/dispatch_dfa.cuh +++ b/cpp/src/io/fst/dispatch_dfa.cuh @@ -119,7 +119,7 @@ struct DispatchFSM : DeviceFSMPolicy { TransducedIndexOutItT transduced_out_idx_it; TransducedCountOutItT d_num_transduced_out_it; cudaStream_t stream; - int ptx_version; + int const ptx_version; //------------------------------------------------------------------------------ // CONSTRUCTOR @@ -422,7 +422,7 @@ struct DispatchFSM : DeviceFSMPolicy { // COMPUTE STATE-TRANSITION VECTORS //------------------------------------------------------------------------------ ScanTileStateT stv_tile_state; - if constexpr(SINGLE_PASS_STV) { + if constexpr (SINGLE_PASS_STV) { // Construct the tile status (aliases memory internally et al.) error = stv_tile_state.Init( num_blocks, allocations[MEM_SINGLE_PASS_STV], allocation_sizes[MEM_SINGLE_PASS_STV]); diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index f9619c82fe8..ed5948249d4 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -35,56 +35,55 @@ namespace detail { * @tparam BackingFragmentT The data type that is holding the fragments */ template <uint32_t NUM_ITEMS, uint32_t MAX_ITEM_VALUE, typename BackingFragmentT = uint32_t> -struct MultiFragmentInRegArray { - /// [b] Minimum number of bits required to represent all values from [0, MAX_ITEM_VALUE] +class MultiFragmentInRegArray { + private: + /// Minimum number of bits required to represent all values from [0, MAX_ITEM_VALUE] static constexpr uint32_t MIN_BITS_PER_ITEM = (MAX_ITEM_VALUE == 0) ? 1 : cub::Log2<(MAX_ITEM_VALUE + 1)>::VALUE; /// Number of bits that each fragment can store static constexpr uint32_t NUM_BITS_PER_FRAGMENT = sizeof(BackingFragmentT) * 8; - /// [a] The number of bits per fragment per item in the array + /// The number of bits per fragment per item in the array static constexpr uint32_t AVAIL_BITS_PER_FRAG_ITEM = NUM_BITS_PER_FRAGMENT / NUM_ITEMS; - /// [k] The number of bits per item per fragment to be a power of two to avoid costly integer + /// The number of bits per item per fragment to be a power of two to avoid costly integer /// multiplication - /// TODO: specialise for VOLTA and later architectures that have efficient integer multiplication static constexpr uint32_t BITS_PER_FRAG_ITEM = 0x01U << (cub::Log2<(AVAIL_BITS_PER_FRAG_ITEM + 1)>::VALUE - 1); - static constexpr uint32_t LOG2_BITS_PER_FRAG_ITEM = cub::Log2<BITS_PER_FRAG_ITEM>::VALUE; - // [f] Number of fragments required to store and to reconstruct an item + // Number of fragments required to store and to reconstruct each item static constexpr uint32_t FRAGMENTS_PER_ITEM = (MIN_BITS_PER_ITEM + BITS_PER_FRAG_ITEM - 1) / BITS_PER_FRAG_ITEM; //------------------------------------------------------------------------------ - // MEMBER VARIABLES + // HELPER FUNCTIONS //------------------------------------------------------------------------------ - __device__ __host__ __forceinline__ unsigned int bfe(const unsigned int& data, - unsigned int bit_start, - unsigned int num_bits) const + __device__ __host__ __forceinline__ uint32_t bfe(const uint32_t& data, + uint32_t bit_start, + uint32_t num_bits) const { #if CUB_PTX_ARCH > 0 return cub::BFE(data, bit_start, num_bits); #else - const unsigned int MASK = (1 << num_bits) - 1; + const uint32_t MASK = (1 << num_bits) - 1; return (data >> bit_start) & MASK; #endif } - __device__ __host__ __forceinline__ void bfi(unsigned int& data, - unsigned int bits, - unsigned int bit_start, - unsigned int num_bits) const + __device__ __host__ __forceinline__ void bfi(uint32_t& data, + uint32_t bits, + uint32_t bit_start, + uint32_t num_bits) const { #if CUB_PTX_ARCH > 0 cub::BFI(data, data, bits, bit_start, num_bits); #else - unsigned int x = bits << bit_start; - unsigned int y = data; - unsigned int MASK_X = ((1 << num_bits) - 1) << bit_start; - unsigned int MASK_Y = ~MASK_X; - data = (y & MASK_Y) | (x & MASK_X); + uint32_t x = bits << bit_start; + uint32_t y = data; + uint32_t MASK_X = ((1 << num_bits) - 1) << bit_start; + uint32_t MASK_Y = ~MASK_X; + data = (y & MASK_Y) | (x & MASK_X); #endif } @@ -93,6 +92,7 @@ struct MultiFragmentInRegArray { //------------------------------------------------------------------------------ // ACCESSORS //------------------------------------------------------------------------------ + public: __host__ __device__ __forceinline__ uint32_t Get(int32_t index) const { uint32_t val = 0; From 39a6b65c9fc4ad12d33155b54c8373b98de2de43 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 25 Apr 2022 09:59:37 -0700 Subject: [PATCH 03/81] refactored lookup tables --- cpp/src/io/fst/agent_dfa.cuh | 3 - cpp/src/io/fst/device_dfa.cuh | 192 +++++++++++++-------------- cpp/src/io/fst/symbol_lut.cuh | 94 +++++-------- cpp/src/io/fst/transition_table.cuh | 109 +++++---------- cpp/src/io/fst/translation_table.cuh | 123 +++++++---------- cpp/tests/io/fst/fst_test.cu | 5 +- 6 files changed, 209 insertions(+), 317 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index 0611973f78c..3bc59160696 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -152,8 +152,6 @@ class StateVectorTransitionOp : public StateTransitionCallbackOp { __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const& character_index, SymbolIndexT const read_symbol_id) const { - using TransitionVectorT = typename TransitionTableT::TransitionVectorT; - for (int32_t i = 0; i < NUM_INSTANCES; ++i) { state_vector.Set(i, transition_table(state_vector.Get(i), read_symbol_id)); } @@ -185,7 +183,6 @@ struct StateTransitionOp { __host__ __device__ __forceinline__ void ReadSymbol(const CharIndexT& character_index, const SymbolIndexT& read_symbol_id) { - using TransitionVectorT = typename TransitionTableT::TransitionVectorT; old_state_vector = state_vector; state_vector.Set(0, transition_table(state_vector.Get(0), read_symbol_id)); callback_op.ReadSymbol(character_index, old_state_vector, state_vector, read_symbol_id); diff --git a/cpp/src/io/fst/device_dfa.cuh b/cpp/src/io/fst/device_dfa.cuh index 795c4c98bec..b12283a9673 100644 --- a/cpp/src/io/fst/device_dfa.cuh +++ b/cpp/src/io/fst/device_dfa.cuh @@ -15,8 +15,9 @@ */ #pragma once -#include "cub/util_type.cuh" #include "dispatch_dfa.cuh" + +#include <io/utilities/hostdevice_vector.hpp> #include <src/io/fst/symbol_lut.cuh> #include <src/io/fst/transition_table.cuh> #include <src/io/fst/translation_table.cuh> @@ -95,140 +96,121 @@ cudaError_t DeviceTransduce(void* d_temp_storage, stream); } -/** - * @brief Helper class to facilitate the specification and instantiation of a DFA (i.e., the - * transition table and its number of states, the mapping of symbols to symbol groups, and the - * translation table that specifies which state transitions cause which output to be written). - * - * @tparam OutSymbolT The symbol type being output by the finite-state transducer - * @tparam NUM_SYMBOLS The number of symbol groups amongst which to differentiate (one dimension of - * the transition table) - * @tparam TT_NUM_STATES The number of states defined by the DFA (the other dimension of the - * transition table) - */ -template <typename OutSymbolT, int32_t NUM_SYMBOLS, int32_t TT_NUM_STATES> -class Dfa { +template <typename SymbolGroupIdLookupT, + typename TransitionTableT, + typename TranslationTableT, + int32_t NUM_STATES> +class dfa_device_view { + private: + using sgid_lut_init_t = typename SymbolGroupIdLookupT::KernelParameter; + using transition_table_init_t = typename TransitionTableT::KernelParameter; + using translation_table_init_t = typename TranslationTableT::KernelParameter; + public: // The maximum number of states supported by this DFA instance // This is a value queried by the DFA simulation algorithm - static constexpr int32_t MAX_NUM_STATES = TT_NUM_STATES; + static constexpr int32_t MAX_NUM_STATES = NUM_STATES; - private: - // Symbol-group id lookup table - using MatcherT = detail::SingleSymbolSmemLUT<char>; - using MatcherInitT = typename MatcherT::KernelParameter; - - // Transition table - using TransitionTableT = detail::TransitionTable<NUM_SYMBOLS + 1, TT_NUM_STATES>; - using TransitionTableInitT = typename TransitionTableT::KernelParameter; - - // Translation lookup table - using OutSymbolOffsetT = uint32_t; - using TransducerTableT = detail::TransducerLookupTable<OutSymbolT, - OutSymbolOffsetT, - NUM_SYMBOLS + 1, - TT_NUM_STATES, - (NUM_SYMBOLS + 1) * TT_NUM_STATES>; - using TransducerTableInitT = typename TransducerTableT::KernelParameter; - - // Private members (passed between host/device) - /// Information to initialize the device-side lookup table that maps symbol -> symbol group id - MatcherInitT symbol_matcher_init; - - /// Information to initialize the device-side transition table - TransitionTableInitT tt_init; - - /// Information to initialize the device-side translation table - TransducerTableInitT tt_out_init; - - public: //--------------------------------------------------------------------- // DEVICE-SIDE MEMBER FUNCTIONS //--------------------------------------------------------------------- - using SymbolGroupStorageT = typename MatcherT::TempStorage; + using SymbolGroupStorageT = typename SymbolGroupIdLookupT::TempStorage; using TransitionTableStorageT = typename TransitionTableT::TempStorage; - using TranslationTableStorageT = typename TransducerTableT::TempStorage; + using TranslationTableStorageT = typename TranslationTableT::TempStorage; __device__ auto InitSymbolGroupLUT(SymbolGroupStorageT& temp_storage) { - return MatcherT(symbol_matcher_init, temp_storage); + return SymbolGroupIdLookupT(*d_sgid_lut_init, temp_storage); } __device__ auto InitTransitionTable(TransitionTableStorageT& temp_storage) { - return TransitionTableT(tt_init, temp_storage); + return TransitionTableT(*d_transition_table_init, temp_storage); } __device__ auto InitTranslationTable(TranslationTableStorageT& temp_storage) { - return TransducerTableT(tt_out_init, temp_storage); + return TranslationTableT(*d_translation_table_init, temp_storage); } - //--------------------------------------------------------------------- - // HOST-SIDE MEMBER FUNCTIONS - //--------------------------------------------------------------------- - template <typename StateIdT, typename SymbolGroupIdItT> - cudaError_t Init(SymbolGroupIdItT const& symbol_vec, - std::vector<std::vector<StateIdT>> const& tt_vec, - std::vector<std::vector<std::vector<OutSymbolT>>> const& out_tt_vec, - cudaStream_t stream = 0) + dfa_device_view(sgid_lut_init_t const* d_sgid_lut_init, + transition_table_init_t const* d_transition_table_init, + translation_table_init_t const* d_translation_table_init) + : d_sgid_lut_init(d_sgid_lut_init), + d_transition_table_init(d_transition_table_init), + d_translation_table_init(d_translation_table_init) { - cudaError_t error = cudaSuccess; - - enum : uint32_t { MEM_SYMBOL_MATCHER = 0, MEM_TT, MEM_OUT_TT, NUM_ALLOCATIONS }; + } - size_t allocation_sizes[NUM_ALLOCATIONS] = {0}; - void* allocations[NUM_ALLOCATIONS] = {0}; + private: + sgid_lut_init_t const* d_sgid_lut_init; + transition_table_init_t const* d_transition_table_init; + translation_table_init_t const* d_translation_table_init; +}; - // Memory requirements: lookup table - error = MatcherT::PrepareLUT( - nullptr, allocation_sizes[MEM_SYMBOL_MATCHER], symbol_vec, symbol_matcher_init); - if (error) return error; +/** + * @brief Helper class to facilitate the specification and instantiation of a DFA (i.e., the + * transition table and its number of states, the mapping of symbols to symbol groups, and the + * translation table that specifies which state transitions cause which output to be written). + * + * @tparam OutSymbolT The symbol type being output by the finite-state transducer + * @tparam NUM_SYMBOLS The number of symbol groups amongst which to differentiate (one dimension of + * the transition table) + * @tparam NUM_STATES The number of states defined by the DFA (the other dimension of the + * transition table) + */ +template <typename OutSymbolT, int32_t NUM_SYMBOLS, int32_t NUM_STATES> +class Dfa { + public: + // The maximum number of states supported by this DFA instance + // This is a value queried by the DFA simulation algorithm + static constexpr int32_t MAX_NUM_STATES = NUM_STATES; - // Memory requirements: transition table - error = - TransitionTableT::CreateTransitionTable(nullptr, allocation_sizes[MEM_TT], tt_vec, tt_init); - if (error) return error; + private: + // Symbol-group id lookup table + using SymbolGroupIdLookupT = detail::SingleSymbolSmemLUT<char>; + using SymbolGroupIdInitT = typename SymbolGroupIdLookupT::KernelParameter; - // Memory requirements: transducer table - error = TransducerTableT::CreateTransitionTable( - nullptr, allocation_sizes[MEM_OUT_TT], out_tt_vec, tt_out_init); - if (error) return error; + // Transition table + using TransitionTableT = detail::TransitionTable<NUM_SYMBOLS + 1, NUM_STATES>; + using TransitionTableInitT = typename TransitionTableT::KernelParameter; - // Memory requirements: total memory - size_t temp_storage_bytes = 0; - error = cub::AliasTemporaries(nullptr, temp_storage_bytes, allocations, allocation_sizes); - if (error) return error; + // Translation lookup table + using OutSymbolOffsetT = uint32_t; + using TranslationTableT = detail::TransducerLookupTable<OutSymbolT, + OutSymbolOffsetT, + NUM_SYMBOLS + 1, + NUM_STATES, + (NUM_SYMBOLS + 1) * NUM_STATES>; + using TranslationTableInitT = typename TranslationTableT::KernelParameter; + + auto get_device_view() + { + return dfa_device_view<SymbolGroupIdLookupT, TransitionTableT, TranslationTableT, NUM_STATES>{ + sgid_init.d_begin(), transition_table_init.d_begin(), translation_table_init.d_begin()}; + } - // Allocate memory - void* d_temp_storage = nullptr; - error = cudaMalloc(&d_temp_storage, temp_storage_bytes); - if (error) return error; + public: + template <typename StateIdT, typename SymbolGroupIdItT> + Dfa(SymbolGroupIdItT const& symbol_vec, + std::vector<std::vector<StateIdT>> const& tt_vec, + std::vector<std::vector<std::vector<OutSymbolT>>> const& out_tt_vec, + cudaStream_t stream) + { + constexpr std::size_t single_item = 1; - // Alias memory - error = - cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); - if (error) return error; + sgid_init = hostdevice_vector<SymbolGroupIdInitT>{single_item, stream}; + transition_table_init = hostdevice_vector<TransitionTableInitT>{single_item, stream}; + translation_table_init = hostdevice_vector<TranslationTableInitT>{single_item, stream}; - // Initialize symbol group lookup table - error = MatcherT::PrepareLUT(allocations[MEM_SYMBOL_MATCHER], - allocation_sizes[MEM_SYMBOL_MATCHER], - symbol_vec, - symbol_matcher_init, - stream); - if (error) return error; + // Initialize symbol group id lookup table + SymbolGroupIdLookupT::InitDeviceSymbolGroupIdLut(sgid_init, symbol_vec, stream); // Initialize state transition table - error = TransitionTableT::CreateTransitionTable( - allocations[MEM_TT], allocation_sizes[MEM_TT], tt_vec, tt_init, stream); - if (error) return error; + TransitionTableT::InitDeviceTransitionTable(transition_table_init, tt_vec, stream); // Initialize finite-state transducer lookup table - error = TransducerTableT::CreateTransitionTable( - allocations[MEM_OUT_TT], allocation_sizes[MEM_OUT_TT], out_tt_vec, tt_out_init, stream); - if (error) return error; - - return error; + TranslationTableT::InitDeviceTranslationTable(translation_table_init, out_tt_vec, stream); } template <typename SymbolT, @@ -248,7 +230,7 @@ class Dfa { { return DeviceTransduce(d_temp_storage, temp_storage_bytes, - *this, + this->get_device_view(), d_chars, num_chars, d_out_it, @@ -257,8 +239,12 @@ class Dfa { seed_state, stream); } -}; + private: + hostdevice_vector<SymbolGroupIdInitT> sgid_init{}; + hostdevice_vector<TransitionTableInitT> transition_table_init{}; + hostdevice_vector<TranslationTableInitT> translation_table_init{}; +}; } // namespace fst } // namespace io } // namespace cudf diff --git a/cpp/src/io/fst/symbol_lut.cuh b/cpp/src/io/fst/symbol_lut.cuh index 08d5f4db58d..abf71a7fbea 100644 --- a/cpp/src/io/fst/symbol_lut.cuh +++ b/cpp/src/io/fst/symbol_lut.cuh @@ -16,6 +16,9 @@ #pragma once +#include <cudf/types.hpp> +#include <io/utilities/hostdevice_vector.hpp> + #include <cub/cub.cuh> #include <algorithm> @@ -34,38 +37,29 @@ namespace detail { * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id */ template <typename SymbolT> -struct SingleSymbolSmemLUT { - //------------------------------------------------------------------------------ - // DEFAULT TYPEDEFS - //------------------------------------------------------------------------------ +class SingleSymbolSmemLUT { + private: // Type used for representing a symbol group id (i.e., what we return for a given symbol) using SymbolGroupIdT = uint8_t; - //------------------------------------------------------------------------------ - // DERIVED CONFIGURATIONS - //------------------------------------------------------------------------------ /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); - //------------------------------------------------------------------------------ - // TYPEDEFS - //------------------------------------------------------------------------------ - struct _TempStorage { - // d_match_meta_data[symbol] -> symbol group index - SymbolGroupIdT match_meta_data[NUM_ENTRIES_PER_LUT]; + // sym_to_sgid[symbol] -> symbol group index + SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; }; + public: struct KernelParameter { - // d_match_meta_data[min(symbol,num_valid_entries)] -> symbol group index - SymbolGroupIdT num_valid_entries; + // sym_to_sgid[min(symbol,num_valid_entries)] -> symbol group index + SymbolT num_valid_entries; - // d_match_meta_data[symbol] -> symbol group index - SymbolGroupIdT* d_match_meta_data; + // sym_to_sgid[symbol] -> symbol group index + SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; }; - struct TempStorage : cub::Uninitialized<_TempStorage> { - }; + using TempStorage = cub::Uninitialized<_TempStorage>; //------------------------------------------------------------------------------ // HELPER METHODS @@ -73,66 +67,48 @@ struct SingleSymbolSmemLUT { /** * @brief * - * @param[in] d_temp_storage Device-side temporary storage that can be used to store the lookup - * table. If no storage is provided it will return the temporary storage requirements in \p - * d_temp_storage_bytes. - * @param[in,out] d_temp_storage_bytes Amount of device-side temporary storage that can be used in - * the number of bytes + * @param[out] sgid_init A hostdevice_vector that will be populated * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols * (characters!) that correspond to the i-th symbol group index - * @param[out] kernel_param The kernel parameter object to be initialized with the given mapping - * of symbols to symbol group ids. * @param[in] stream The stream that shall be used to cudaMemcpyAsync the lookup table * @return */ template <typename SymbolGroupItT> - __host__ __forceinline__ static cudaError_t PrepareLUT(void* d_temp_storage, - size_t& d_temp_storage_bytes, - SymbolGroupItT const& symbol_strings, - KernelParameter& kernel_param, - cudaStream_t stream = 0) + static void InitDeviceSymbolGroupIdLut(hostdevice_vector<KernelParameter>& sgid_init, + SymbolGroupItT const& symbol_strings, + rmm::cuda_stream_view stream) { // The symbol group index to be returned if none of the given symbols match SymbolGroupIdT no_match_id = symbol_strings.size(); - std::vector<SymbolGroupIdT> lut(NUM_ENTRIES_PER_LUT); + // The symbol with the largest value that is mapped to a symbol group id SymbolGroupIdT max_base_match_val = 0; // Initialize all entries: by default we return the no-match-id - for (uint32_t i = 0; i < NUM_ENTRIES_PER_LUT; ++i) { - lut[i] = no_match_id; - } + std::fill(&sgid_init.host_ptr()->sym_to_sgid[0], + &sgid_init.host_ptr()->sym_to_sgid[NUM_ENTRIES_PER_LUT], + no_match_id); // Set up lookup table uint32_t sg_id = 0; + // Iterate over the symbol groups for (auto const& sg_symbols : symbol_strings) { + // Iterate over all symbols that belong to the current symbol group for (auto const& sg_symbol : sg_symbols) { max_base_match_val = std::max(max_base_match_val, static_cast<SymbolGroupIdT>(sg_symbol)); - lut[sg_symbol] = sg_id; + sgid_init.host_ptr()->sym_to_sgid[static_cast<int32_t>(sg_symbol)] = sg_id; } sg_id++; } - // Initialize the out-of-bounds lookup: d_match_meta_data[max_base_match_val+1] -> no_match_id - lut[max_base_match_val + 1] = no_match_id; + // Initialize the out-of-bounds lookup: sym_to_sgid[max_base_match_val+1] -> no_match_id + sgid_init.host_ptr()->sym_to_sgid[max_base_match_val + 1] = no_match_id; // Alias memory / return memory requiremenets - kernel_param.num_valid_entries = max_base_match_val + 2; - if (d_temp_storage) { - cudaError_t error = cudaMemcpyAsync(d_temp_storage, - lut.data(), - kernel_param.num_valid_entries * sizeof(SymbolGroupIdT), - cudaMemcpyHostToDevice, - stream); - - kernel_param.d_match_meta_data = reinterpret_cast<SymbolGroupIdT*>(d_temp_storage); - return error; - } else { - d_temp_storage_bytes = kernel_param.num_valid_entries * sizeof(SymbolGroupIdT); - return cudaSuccess; - } + // TODO I think this could be +1? + sgid_init.host_ptr()->num_valid_entries = max_base_match_val + 2; - return cudaSuccess; + sgid_init.host_to_device(stream); } //------------------------------------------------------------------------------ @@ -150,29 +126,29 @@ struct SingleSymbolSmemLUT { return private_storage; } - __host__ __device__ __forceinline__ SingleSymbolSmemLUT(KernelParameter const& kernel_param, - TempStorage& temp_storage) + constexpr CUDF_HOST_DEVICE SingleSymbolSmemLUT(KernelParameter const& kernel_param, + TempStorage& temp_storage) : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) { // GPU-side init #if CUB_PTX_ARCH > 0 for (int32_t i = threadIdx.x; i < kernel_param.num_valid_entries; i += blockDim.x) { - this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; } __syncthreads(); #else // CPU-side init for (std::size_t i = 0; i < kernel_param.num_luts; i++) { - this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; } #endif } - __host__ __device__ __forceinline__ int32_t operator()(SymbolT const symbol) const + constexpr CUDF_HOST_DEVICE int32_t operator()(SymbolT const symbol) const { // Look up the symbol group for given symbol - return temp_storage.match_meta_data[min(symbol, num_valid_entries - 1)]; + return temp_storage.sym_to_sgid[min(symbol, num_valid_entries - 1)]; } }; diff --git a/cpp/src/io/fst/transition_table.cuh b/cpp/src/io/fst/transition_table.cuh index 97fef03d8af..5eccb926974 100644 --- a/cpp/src/io/fst/transition_table.cuh +++ b/cpp/src/io/fst/transition_table.cuh @@ -16,6 +16,10 @@ #pragma once +#include <cudf/types.hpp> +#include <cudf/utilities/error.hpp> +#include <io/utilities/hostdevice_vector.hpp> + #include <cub/cub.cuh> #include <cstdint> @@ -25,103 +29,50 @@ namespace io { namespace fst { namespace detail { -template <int MAX_NUM_SYMBOLS, int MAX_NUM_STATES> -struct TransitionTable { - //------------------------------------------------------------------------------ - // DEFAULT TYPEDEFS - //------------------------------------------------------------------------------ +template <int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES> +class TransitionTable { + private: + // Type used using ItemT = char; - struct TransitionVectorWrapper { - const ItemT* data; - - __host__ __device__ TransitionVectorWrapper(const ItemT* data) : data(data) {} - - __host__ __device__ __forceinline__ uint32_t Get(int32_t index) const { return data[index]; } - }; - - //------------------------------------------------------------------------------ - // TYPEDEFS - //------------------------------------------------------------------------------ - using TransitionVectorT = TransitionVectorWrapper; - struct _TempStorage { - // ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; }; - struct TempStorage : cub::Uninitialized<_TempStorage> { - }; + public: + using TempStorage = cub::Uninitialized<_TempStorage>; struct KernelParameter { - ItemT* transitions; + ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; }; - using LoadAliasT = std::uint32_t; - - static constexpr std::size_t NUM_AUX_MEM_BYTES = - CUB_QUOTIENT_CEILING(MAX_NUM_STATES * MAX_NUM_SYMBOLS * sizeof(ItemT), sizeof(LoadAliasT)) * - sizeof(LoadAliasT); - - //------------------------------------------------------------------------------ - // HELPER METHODS - //------------------------------------------------------------------------------ - __host__ static cudaError_t CreateTransitionTable( - void* d_temp_storage, - size_t& temp_storage_bytes, - const std::vector<std::vector<int>>& trans_table, - KernelParameter& kernel_param, - cudaStream_t stream = 0) + static void InitDeviceTransitionTable(hostdevice_vector<KernelParameter>& transition_table_init, + const std::vector<std::vector<int>>& trans_table, + rmm::cuda_stream_view stream) { - if (!d_temp_storage) { - temp_storage_bytes = NUM_AUX_MEM_BYTES; - return cudaSuccess; - } - - // trans_vectors[symbol][state] -> new_state - ItemT trans_vectors[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; - // trans_table[state][symbol] -> new state for (std::size_t state = 0; state < trans_table.size(); ++state) { for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { - trans_vectors[symbol * MAX_NUM_STATES + state] = trans_table[state][symbol]; + transition_table_init.host_ptr()->transitions[symbol * MAX_NUM_STATES + state] = + trans_table[state][symbol]; } } - kernel_param.transitions = static_cast<ItemT*>(d_temp_storage); - // Copy transition table to device - return cudaMemcpyAsync( - d_temp_storage, trans_vectors, NUM_AUX_MEM_BYTES, cudaMemcpyHostToDevice, stream); + transition_table_init.host_to_device(stream); } - //------------------------------------------------------------------------------ - // MEMBER VARIABLES - //------------------------------------------------------------------------------ - _TempStorage& temp_storage; - - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - return private_storage; - } - - //------------------------------------------------------------------------------ - // CONSTRUCTOR - //------------------------------------------------------------------------------ - __host__ __device__ __forceinline__ TransitionTable(const KernelParameter& kernel_param, - TempStorage& temp_storage) + constexpr CUDF_HOST_DEVICE TransitionTable(const KernelParameter& kernel_param, + TempStorage& temp_storage) : temp_storage(temp_storage.Alias()) { #if CUB_PTX_ARCH > 0 - for (int i = threadIdx.x; i < CUB_QUOTIENT_CEILING(NUM_AUX_MEM_BYTES, sizeof(LoadAliasT)); - i += blockDim.x) { - reinterpret_cast<LoadAliasT*>(this->temp_storage.transitions)[i] = - reinterpret_cast<LoadAliasT*>(kernel_param.transitions)[i]; + for (int i = threadIdx.x; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i += blockDim.x) { + this->temp_storage.transitions[i] = kernel_param.transitions[i]; } __syncthreads(); #else - for (int i = 0; i < kernel_param.num_luts; i++) { + for (int i = 0; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i++) { this->temp_storage.transitions[i] = kernel_param.transitions[i]; } #endif @@ -136,11 +87,21 @@ struct TransitionTable { * @return */ template <typename StateIndexT, typename SymbolIndexT> - __host__ __device__ __forceinline__ int32_t operator()(StateIndexT state_id, - SymbolIndexT match_id) const + constexpr CUDF_HOST_DEVICE int32_t operator()(StateIndexT const state_id, + SymbolIndexT const match_id) const { return temp_storage.transitions[match_id * MAX_NUM_STATES + state_id]; - } + } + + private: + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + + return private_storage; + } }; } // namespace detail diff --git a/cpp/src/io/fst/translation_table.cuh b/cpp/src/io/fst/translation_table.cuh index bfbfd41e3f0..89da994606c 100644 --- a/cpp/src/io/fst/translation_table.cuh +++ b/cpp/src/io/fst/translation_table.cuh @@ -16,7 +16,12 @@ #pragma once -#include "in_reg_array.cuh" +#include <algorithm> +#include <cudf/types.hpp> +#include <cudf/utilities/error.hpp> +#include <io/utilities/hostdevice_vector.hpp> + +#include "rmm/device_uvector.hpp" #include <cub/cub.cuh> @@ -28,10 +33,10 @@ namespace fst { namespace detail { /** - * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols to - * output + * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols + * that the finite-state transducer is supposed to output for each transition * - * @tparam OutSymbolT The symbol type being returned + * @tparam OutSymbolT The symbol type being output * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support @@ -42,57 +47,35 @@ template <typename OutSymbolT, int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES, int32_t MAX_TABLE_SIZE = (MAX_NUM_SYMBOLS * MAX_NUM_STATES)> -struct TransducerLookupTable { - //------------------------------------------------------------------------------ - // TYPEDEFS - //------------------------------------------------------------------------------ +class TransducerLookupTable { + private: struct _TempStorage { OutSymbolOffsetT out_offset[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; OutSymbolT out_symbols[MAX_TABLE_SIZE]; }; - struct TempStorage : cub::Uninitialized<_TempStorage> { - }; + public: + using TempStorage = cub::Uninitialized<_TempStorage>; struct KernelParameter { - OutSymbolOffsetT* d_trans_offsets; - OutSymbolT* d_out_symbols; + OutSymbolOffsetT d_out_offsets[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; + OutSymbolT d_out_symbols[MAX_TABLE_SIZE]; }; - //------------------------------------------------------------------------------ - // HELPER METHODS - //------------------------------------------------------------------------------ - __host__ static cudaError_t CreateTransitionTable( - void* d_temp_storage, - size_t& temp_storage_bytes, - const std::vector<std::vector<std::vector<OutSymbolT>>>& trans_table, - KernelParameter& kernel_param, - cudaStream_t stream = 0) + /** + * @brief Initializes the translation table (both the host and device parts) + */ + static void InitDeviceTranslationTable( + hostdevice_vector<KernelParameter>& translation_table_init, + std::vector<std::vector<std::vector<OutSymbolT>>> const& trans_table, + rmm::cuda_stream_view stream) { - enum { MEM_OFFSETS = 0, MEM_OUT_SYMBOLS, NUM_ALLOCATIONS }; - - size_t allocation_sizes[NUM_ALLOCATIONS] = {}; - void* allocations[NUM_ALLOCATIONS] = {}; - allocation_sizes[MEM_OFFSETS] = - (MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1) * sizeof(OutSymbolOffsetT); - allocation_sizes[MEM_OUT_SYMBOLS] = MAX_TABLE_SIZE * sizeof(OutSymbolT); - - // Alias the temporary allocations from the single storage blob (or compute the necessary size - // of the blob) - cudaError_t error = - cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); - if (error) return error; - - // Return if the caller is simply requesting the size of the storage allocation - if (d_temp_storage == nullptr) return cudaSuccess; - std::vector<OutSymbolT> out_symbols; out_symbols.reserve(MAX_TABLE_SIZE); std::vector<OutSymbolOffsetT> out_symbol_offsets; out_symbol_offsets.reserve(MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1); out_symbol_offsets.push_back(0); - int st = 0; // Iterate over the states in the transition table for (auto const& state_trans : trans_table) { uint32_t num_added = 0; @@ -103,7 +86,6 @@ struct TransducerLookupTable { out_symbol_offsets.push_back(out_symbols.size()); num_added++; } - st++; // Copy the last offset for all symbols (to guarantee a proper lookup for omitted symbols of // this state) @@ -115,30 +97,21 @@ struct TransducerLookupTable { } // Check whether runtime-provided table size exceeds the compile-time given max. table size - if (out_symbols.size() > MAX_TABLE_SIZE) { return cudaErrorInvalidValue; } - - kernel_param.d_trans_offsets = static_cast<OutSymbolOffsetT*>(allocations[MEM_OFFSETS]); - kernel_param.d_out_symbols = static_cast<OutSymbolT*>(allocations[MEM_OUT_SYMBOLS]); - - // Copy out symbols - error = cudaMemcpyAsync(kernel_param.d_trans_offsets, - out_symbol_offsets.data(), - out_symbol_offsets.size() * sizeof(out_symbol_offsets[0]), - cudaMemcpyHostToDevice, - stream); - if (error) { return error; } - - // Copy offsets into output symbols - return cudaMemcpyAsync(kernel_param.d_out_symbols, - out_symbols.data(), - out_symbols.size() * sizeof(out_symbols[0]), - cudaMemcpyHostToDevice, - stream); + if (out_symbols.size() > MAX_TABLE_SIZE) { CUDF_FAIL("Unsupported translation table"); } + + // Prepare host-side data to be copied and passed to the device + std::copy(std::cbegin(out_symbol_offsets), + std::cend(out_symbol_offsets), + translation_table_init.host_ptr()->d_out_offsets); + std::copy(std::cbegin(out_symbols), + std::cend(out_symbols), + translation_table_init.host_ptr()->d_out_symbols); + + // Copy data to device + translation_table_init.host_to_device(stream); } - //------------------------------------------------------------------------------ - // MEMBER VARIABLES - //------------------------------------------------------------------------------ + private: _TempStorage& temp_storage; __device__ __forceinline__ _TempStorage& PrivateStorage() @@ -147,17 +120,19 @@ struct TransducerLookupTable { return private_storage; } - //------------------------------------------------------------------------------ - // CONSTRUCTOR - //------------------------------------------------------------------------------ - __host__ __device__ __forceinline__ TransducerLookupTable(const KernelParameter& kernel_param, - TempStorage& temp_storage) + public: + /** + * @brief Synchronizes the thread block, if called from device, and, hence, requires all threads + * of the thread block to call the constructor + */ + CUDF_HOST_DEVICE TransducerLookupTable(KernelParameter const& kernel_param, + TempStorage& temp_storage) : temp_storage(temp_storage.Alias()) { constexpr uint32_t num_offsets = MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1; #if CUB_PTX_ARCH > 0 for (int i = threadIdx.x; i < num_offsets; i += blockDim.x) { - this->temp_storage.out_offset[i] = kernel_param.d_trans_offsets[i]; + this->temp_storage.out_offset[i] = kernel_param.d_out_offsets[i]; } // Make sure all threads in the block can read out_symbol_offsets[num_offsets - 1] from shared // memory @@ -168,7 +143,7 @@ struct TransducerLookupTable { __syncthreads(); #else for (int i = 0; i < num_offsets; i++) { - this->temp_storage.out_symbol_offsets[i] = kernel_param.d_trans_offsets[i]; + this->temp_storage.out_symbol_offsets[i] = kernel_param.d_out_offsets[i]; } for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; @@ -177,17 +152,17 @@ struct TransducerLookupTable { } template <typename StateIndexT, typename SymbolIndexT, typename RelativeOffsetT> - __host__ __device__ __forceinline__ OutSymbolT operator()(StateIndexT state_id, - SymbolIndexT match_id, - RelativeOffsetT relative_offset) const + constexpr CUDF_HOST_DEVICE OutSymbolT operator()(StateIndexT const state_id, + SymbolIndexT const match_id, + RelativeOffsetT const relative_offset) const { auto offset = temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id] + relative_offset; return temp_storage.out_symbols[offset]; } template <typename StateIndexT, typename SymbolIndexT> - __host__ __device__ __forceinline__ OutSymbolOffsetT operator()(StateIndexT state_id, - SymbolIndexT match_id) const + constexpr CUDF_HOST_DEVICE OutSymbolOffsetT operator()(StateIndexT const state_id, + SymbolIndexT const match_id) const { return temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id + 1] - temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id]; diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 26bb9d47dca..29c93a6f3bb 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -224,10 +224,7 @@ TEST_F(FstTest, GroundTruth) d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); // Run algorithm - DfaFstT parser; - - // Initialize DFA - ASSERT_CUDA_SUCCEEDED(parser.Init(pda_sgs, pda_state_tt, pda_out_tt, stream)); + DfaFstT parser{pda_sgs, pda_state_tt, pda_out_tt, stream}; std::size_t temp_storage_bytes = 0; From 239f138d78cc12af8607f1feb7d7ec4bec2f58fc Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 25 Apr 2022 12:17:08 -0700 Subject: [PATCH 04/81] put lookup tables into their own cudf file --- cpp/src/io/fst/device_dfa.cuh | 152 -------- cpp/src/io/fst/lookup_tables.cuh | 519 +++++++++++++++++++++++++++ cpp/src/io/fst/symbol_lut.cuh | 158 -------- cpp/src/io/fst/transition_table.cuh | 110 ------ cpp/src/io/fst/translation_table.cuh | 175 --------- cpp/tests/io/fst/fst_test.cu | 4 +- 6 files changed, 521 insertions(+), 597 deletions(-) create mode 100644 cpp/src/io/fst/lookup_tables.cuh delete mode 100644 cpp/src/io/fst/symbol_lut.cuh delete mode 100644 cpp/src/io/fst/transition_table.cuh delete mode 100644 cpp/src/io/fst/translation_table.cuh diff --git a/cpp/src/io/fst/device_dfa.cuh b/cpp/src/io/fst/device_dfa.cuh index b12283a9673..d3f0e8be213 100644 --- a/cpp/src/io/fst/device_dfa.cuh +++ b/cpp/src/io/fst/device_dfa.cuh @@ -18,9 +18,6 @@ #include "dispatch_dfa.cuh" #include <io/utilities/hostdevice_vector.hpp> -#include <src/io/fst/symbol_lut.cuh> -#include <src/io/fst/transition_table.cuh> -#include <src/io/fst/translation_table.cuh> #include <cstdint> @@ -96,155 +93,6 @@ cudaError_t DeviceTransduce(void* d_temp_storage, stream); } -template <typename SymbolGroupIdLookupT, - typename TransitionTableT, - typename TranslationTableT, - int32_t NUM_STATES> -class dfa_device_view { - private: - using sgid_lut_init_t = typename SymbolGroupIdLookupT::KernelParameter; - using transition_table_init_t = typename TransitionTableT::KernelParameter; - using translation_table_init_t = typename TranslationTableT::KernelParameter; - - public: - // The maximum number of states supported by this DFA instance - // This is a value queried by the DFA simulation algorithm - static constexpr int32_t MAX_NUM_STATES = NUM_STATES; - - //--------------------------------------------------------------------- - // DEVICE-SIDE MEMBER FUNCTIONS - //--------------------------------------------------------------------- - using SymbolGroupStorageT = typename SymbolGroupIdLookupT::TempStorage; - using TransitionTableStorageT = typename TransitionTableT::TempStorage; - using TranslationTableStorageT = typename TranslationTableT::TempStorage; - - __device__ auto InitSymbolGroupLUT(SymbolGroupStorageT& temp_storage) - { - return SymbolGroupIdLookupT(*d_sgid_lut_init, temp_storage); - } - - __device__ auto InitTransitionTable(TransitionTableStorageT& temp_storage) - { - return TransitionTableT(*d_transition_table_init, temp_storage); - } - - __device__ auto InitTranslationTable(TranslationTableStorageT& temp_storage) - { - return TranslationTableT(*d_translation_table_init, temp_storage); - } - - dfa_device_view(sgid_lut_init_t const* d_sgid_lut_init, - transition_table_init_t const* d_transition_table_init, - translation_table_init_t const* d_translation_table_init) - : d_sgid_lut_init(d_sgid_lut_init), - d_transition_table_init(d_transition_table_init), - d_translation_table_init(d_translation_table_init) - { - } - - private: - sgid_lut_init_t const* d_sgid_lut_init; - transition_table_init_t const* d_transition_table_init; - translation_table_init_t const* d_translation_table_init; -}; - -/** - * @brief Helper class to facilitate the specification and instantiation of a DFA (i.e., the - * transition table and its number of states, the mapping of symbols to symbol groups, and the - * translation table that specifies which state transitions cause which output to be written). - * - * @tparam OutSymbolT The symbol type being output by the finite-state transducer - * @tparam NUM_SYMBOLS The number of symbol groups amongst which to differentiate (one dimension of - * the transition table) - * @tparam NUM_STATES The number of states defined by the DFA (the other dimension of the - * transition table) - */ -template <typename OutSymbolT, int32_t NUM_SYMBOLS, int32_t NUM_STATES> -class Dfa { - public: - // The maximum number of states supported by this DFA instance - // This is a value queried by the DFA simulation algorithm - static constexpr int32_t MAX_NUM_STATES = NUM_STATES; - - private: - // Symbol-group id lookup table - using SymbolGroupIdLookupT = detail::SingleSymbolSmemLUT<char>; - using SymbolGroupIdInitT = typename SymbolGroupIdLookupT::KernelParameter; - - // Transition table - using TransitionTableT = detail::TransitionTable<NUM_SYMBOLS + 1, NUM_STATES>; - using TransitionTableInitT = typename TransitionTableT::KernelParameter; - - // Translation lookup table - using OutSymbolOffsetT = uint32_t; - using TranslationTableT = detail::TransducerLookupTable<OutSymbolT, - OutSymbolOffsetT, - NUM_SYMBOLS + 1, - NUM_STATES, - (NUM_SYMBOLS + 1) * NUM_STATES>; - using TranslationTableInitT = typename TranslationTableT::KernelParameter; - - auto get_device_view() - { - return dfa_device_view<SymbolGroupIdLookupT, TransitionTableT, TranslationTableT, NUM_STATES>{ - sgid_init.d_begin(), transition_table_init.d_begin(), translation_table_init.d_begin()}; - } - - public: - template <typename StateIdT, typename SymbolGroupIdItT> - Dfa(SymbolGroupIdItT const& symbol_vec, - std::vector<std::vector<StateIdT>> const& tt_vec, - std::vector<std::vector<std::vector<OutSymbolT>>> const& out_tt_vec, - cudaStream_t stream) - { - constexpr std::size_t single_item = 1; - - sgid_init = hostdevice_vector<SymbolGroupIdInitT>{single_item, stream}; - transition_table_init = hostdevice_vector<TransitionTableInitT>{single_item, stream}; - translation_table_init = hostdevice_vector<TranslationTableInitT>{single_item, stream}; - - // Initialize symbol group id lookup table - SymbolGroupIdLookupT::InitDeviceSymbolGroupIdLut(sgid_init, symbol_vec, stream); - - // Initialize state transition table - TransitionTableT::InitDeviceTransitionTable(transition_table_init, tt_vec, stream); - - // Initialize finite-state transducer lookup table - TranslationTableT::InitDeviceTranslationTable(translation_table_init, out_tt_vec, stream); - } - - template <typename SymbolT, - typename TransducedOutItT, - typename TransducedIndexOutItT, - typename TransducedCountOutItT, - typename OffsetT> - cudaError_t Transduce(void* d_temp_storage, - size_t& temp_storage_bytes, - SymbolT const* d_chars, - OffsetT num_chars, - TransducedOutItT d_out_it, - TransducedIndexOutItT d_out_idx_it, - TransducedCountOutItT d_num_transduced_out_it, - const uint32_t seed_state = 0, - cudaStream_t stream = 0) - { - return DeviceTransduce(d_temp_storage, - temp_storage_bytes, - this->get_device_view(), - d_chars, - num_chars, - d_out_it, - d_out_idx_it, - d_num_transduced_out_it, - seed_state, - stream); - } - - private: - hostdevice_vector<SymbolGroupIdInitT> sgid_init{}; - hostdevice_vector<TransitionTableInitT> transition_table_init{}; - hostdevice_vector<TranslationTableInitT> translation_table_init{}; -}; } // namespace fst } // namespace io } // namespace cudf diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh new file mode 100644 index 00000000000..58853919b69 --- /dev/null +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -0,0 +1,519 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <cudf/types.hpp> +#include <io/utilities/hostdevice_vector.hpp> +#include <io/fst/device_dfa.cuh> + +#include <cub/cub.cuh> + +#include <algorithm> +#include <cstdint> +#include <vector> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +/** + * @brief Class template that can be plugged into the finite-state machine to look up the symbol + * group index for a given symbol. Class template does not support multi-symbol lookups (i.e., no + * look-ahead). + * + * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id + */ +template <typename SymbolT> +class SingleSymbolSmemLUT { + private: + // Type used for representing a symbol group id (i.e., what we return for a given symbol) + using SymbolGroupIdT = uint8_t; + + /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) + static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); + + struct _TempStorage { + // sym_to_sgid[symbol] -> symbol group index + SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; + }; + + public: + struct KernelParameter { + // sym_to_sgid[min(symbol,num_valid_entries)] -> symbol group index + SymbolT num_valid_entries; + + // sym_to_sgid[symbol] -> symbol group index + SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; + }; + + using TempStorage = cub::Uninitialized<_TempStorage>; + + //------------------------------------------------------------------------------ + // HELPER METHODS + //------------------------------------------------------------------------------ + /** + * @brief + * + * @param[out] sgid_init A hostdevice_vector that will be populated + * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols + * (characters!) that correspond to the i-th symbol group index + * @param[in] stream The stream that shall be used to cudaMemcpyAsync the lookup table + * @return + */ + template <typename SymbolGroupItT> + static void InitDeviceSymbolGroupIdLut(hostdevice_vector<KernelParameter>& sgid_init, + SymbolGroupItT const& symbol_strings, + rmm::cuda_stream_view stream) + { + // The symbol group index to be returned if none of the given symbols match + SymbolGroupIdT no_match_id = symbol_strings.size(); + + // The symbol with the largest value that is mapped to a symbol group id + SymbolGroupIdT max_base_match_val = 0; + + // Initialize all entries: by default we return the no-match-id + std::fill(&sgid_init.host_ptr()->sym_to_sgid[0], + &sgid_init.host_ptr()->sym_to_sgid[NUM_ENTRIES_PER_LUT], + no_match_id); + + // Set up lookup table + uint32_t sg_id = 0; + // Iterate over the symbol groups + for (auto const& sg_symbols : symbol_strings) { + // Iterate over all symbols that belong to the current symbol group + for (auto const& sg_symbol : sg_symbols) { + max_base_match_val = std::max(max_base_match_val, static_cast<SymbolGroupIdT>(sg_symbol)); + sgid_init.host_ptr()->sym_to_sgid[static_cast<int32_t>(sg_symbol)] = sg_id; + } + sg_id++; + } + + // Initialize the out-of-bounds lookup: sym_to_sgid[max_base_match_val+1] -> no_match_id + sgid_init.host_ptr()->sym_to_sgid[max_base_match_val + 1] = no_match_id; + + // Alias memory / return memory requiremenets + // TODO I think this could be +1? + sgid_init.host_ptr()->num_valid_entries = max_base_match_val + 2; + + sgid_init.host_to_device(stream); + } + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + SymbolGroupIdT num_valid_entries; + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + constexpr CUDF_HOST_DEVICE SingleSymbolSmemLUT(KernelParameter const& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) + { + // GPU-side init +#if CUB_PTX_ARCH > 0 + for (int32_t i = threadIdx.x; i < kernel_param.num_valid_entries; i += blockDim.x) { + this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; + } + __syncthreads(); + +#else + // CPU-side init + for (std::size_t i = 0; i < kernel_param.num_luts; i++) { + this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; + } +#endif + } + + constexpr CUDF_HOST_DEVICE int32_t operator()(SymbolT const symbol) const + { + // Look up the symbol group for given symbol + return temp_storage.sym_to_sgid[min(symbol, num_valid_entries - 1)]; + } +}; + +template <int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES> +class TransitionTable { + private: + // Type used + using ItemT = char; + + struct _TempStorage { + ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; + }; + + public: + using TempStorage = cub::Uninitialized<_TempStorage>; + + struct KernelParameter { + ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; + }; + + static void InitDeviceTransitionTable(hostdevice_vector<KernelParameter>& transition_table_init, + const std::vector<std::vector<int>>& trans_table, + rmm::cuda_stream_view stream) + { + // trans_table[state][symbol] -> new state + for (std::size_t state = 0; state < trans_table.size(); ++state) { + for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { + transition_table_init.host_ptr()->transitions[symbol * MAX_NUM_STATES + state] = + trans_table[state][symbol]; + } + } + + // Copy transition table to device + transition_table_init.host_to_device(stream); + } + + constexpr CUDF_HOST_DEVICE TransitionTable(const KernelParameter& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()) + { +#if CUB_PTX_ARCH > 0 + for (int i = threadIdx.x; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i += blockDim.x) { + this->temp_storage.transitions[i] = kernel_param.transitions[i]; + } + __syncthreads(); +#else + for (int i = 0; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i++) { + this->temp_storage.transitions[i] = kernel_param.transitions[i]; + } +#endif + } + + /** + * @brief Returns a random-access iterator to lookup all the state transitions for one specific + * symbol from an arbitrary old_state, i.e., it[old_state] -> new_state. + * + * @param state_id The DFA's current state index from which we'll transition + * @param match_id The symbol group id of the symbol that we just read in + * @return + */ + template <typename StateIndexT, typename SymbolIndexT> + constexpr CUDF_HOST_DEVICE int32_t operator()(StateIndexT const state_id, + SymbolIndexT const match_id) const + { + return temp_storage.transitions[match_id * MAX_NUM_STATES + state_id]; + } + + private: + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + + return private_storage; + } +}; + +template <typename SymbolGroupIdLookupT, + typename TransitionTableT, + typename TranslationTableT, + int32_t NUM_STATES> +class dfa_device_view { + private: + using sgid_lut_init_t = typename SymbolGroupIdLookupT::KernelParameter; + using transition_table_init_t = typename TransitionTableT::KernelParameter; + using translation_table_init_t = typename TranslationTableT::KernelParameter; + + public: + // The maximum number of states supported by this DFA instance + // This is a value queried by the DFA simulation algorithm + static constexpr int32_t MAX_NUM_STATES = NUM_STATES; + + using SymbolGroupStorageT = typename SymbolGroupIdLookupT::TempStorage; + using TransitionTableStorageT = typename TransitionTableT::TempStorage; + using TranslationTableStorageT = typename TranslationTableT::TempStorage; + + __device__ auto InitSymbolGroupLUT(SymbolGroupStorageT& temp_storage) + { + return SymbolGroupIdLookupT(*d_sgid_lut_init, temp_storage); + } + + __device__ auto InitTransitionTable(TransitionTableStorageT& temp_storage) + { + return TransitionTableT(*d_transition_table_init, temp_storage); + } + + __device__ auto InitTranslationTable(TranslationTableStorageT& temp_storage) + { + return TranslationTableT(*d_translation_table_init, temp_storage); + } + + dfa_device_view(sgid_lut_init_t const* d_sgid_lut_init, + transition_table_init_t const* d_transition_table_init, + translation_table_init_t const* d_translation_table_init) + : d_sgid_lut_init(d_sgid_lut_init), + d_transition_table_init(d_transition_table_init), + d_translation_table_init(d_translation_table_init) + { + } + + private: + sgid_lut_init_t const* d_sgid_lut_init; + transition_table_init_t const* d_transition_table_init; + translation_table_init_t const* d_translation_table_init; +}; + +/** + * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols + * that the finite-state transducer is supposed to output for each transition + * + * @tparam OutSymbolT The symbol type being output + * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols + * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition + * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support + * @tparam MAX_TABLE_SIZE The maximum number of items in the lookup table of output symbols + */ +template <typename OutSymbolT, + typename OutSymbolOffsetT, + int32_t MAX_NUM_SYMBOLS, + int32_t MAX_NUM_STATES, + int32_t MAX_TABLE_SIZE = (MAX_NUM_SYMBOLS * MAX_NUM_STATES)> +class TransducerLookupTable { + private: + struct _TempStorage { + OutSymbolOffsetT out_offset[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; + OutSymbolT out_symbols[MAX_TABLE_SIZE]; + }; + + public: + using TempStorage = cub::Uninitialized<_TempStorage>; + + struct KernelParameter { + OutSymbolOffsetT d_out_offsets[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; + OutSymbolT d_out_symbols[MAX_TABLE_SIZE]; + }; + + /** + * @brief Initializes the translation table (both the host and device parts) + */ + static void InitDeviceTranslationTable( + hostdevice_vector<KernelParameter>& translation_table_init, + std::vector<std::vector<std::vector<OutSymbolT>>> const& trans_table, + rmm::cuda_stream_view stream) + { + std::vector<OutSymbolT> out_symbols; + out_symbols.reserve(MAX_TABLE_SIZE); + std::vector<OutSymbolOffsetT> out_symbol_offsets; + out_symbol_offsets.reserve(MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1); + out_symbol_offsets.push_back(0); + + // Iterate over the states in the transition table + for (auto const& state_trans : trans_table) { + uint32_t num_added = 0; + // Iterate over the symbols in the transition table + for (auto const& symbol_out : state_trans) { + // Insert the output symbols for this specific (state, symbol) transition + out_symbols.insert(std::end(out_symbols), std::begin(symbol_out), std::end(symbol_out)); + out_symbol_offsets.push_back(out_symbols.size()); + num_added++; + } + + // Copy the last offset for all symbols (to guarantee a proper lookup for omitted symbols of + // this state) + if (MAX_NUM_SYMBOLS > num_added) { + int32_t count = MAX_NUM_SYMBOLS - num_added; + auto begin_it = std::prev(std::end(out_symbol_offsets)); + std::copy(begin_it, begin_it + count, std::back_inserter(out_symbol_offsets)); + } + } + + // Check whether runtime-provided table size exceeds the compile-time given max. table size + if (out_symbols.size() > MAX_TABLE_SIZE) { CUDF_FAIL("Unsupported translation table"); } + + // Prepare host-side data to be copied and passed to the device + std::copy(std::cbegin(out_symbol_offsets), + std::cend(out_symbol_offsets), + translation_table_init.host_ptr()->d_out_offsets); + std::copy(std::cbegin(out_symbols), + std::cend(out_symbols), + translation_table_init.host_ptr()->d_out_symbols); + + // Copy data to device + translation_table_init.host_to_device(stream); + } + + private: + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + public: + /** + * @brief Synchronizes the thread block, if called from device, and, hence, requires all threads + * of the thread block to call the constructor + */ + CUDF_HOST_DEVICE TransducerLookupTable(KernelParameter const& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()) + { + constexpr uint32_t num_offsets = MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1; +#if CUB_PTX_ARCH > 0 + for (int i = threadIdx.x; i < num_offsets; i += blockDim.x) { + this->temp_storage.out_offset[i] = kernel_param.d_out_offsets[i]; + } + // Make sure all threads in the block can read out_symbol_offsets[num_offsets - 1] from shared + // memory + __syncthreads(); + for (int i = threadIdx.x; i < this->temp_storage.out_offset[num_offsets - 1]; i += blockDim.x) { + this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; + } + __syncthreads(); +#else + for (int i = 0; i < num_offsets; i++) { + this->temp_storage.out_symbol_offsets[i] = kernel_param.d_out_offsets[i]; + } + for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { + this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; + } +#endif + } + + template <typename StateIndexT, typename SymbolIndexT, typename RelativeOffsetT> + constexpr CUDF_HOST_DEVICE OutSymbolT operator()(StateIndexT const state_id, + SymbolIndexT const match_id, + RelativeOffsetT const relative_offset) const + { + auto offset = temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id] + relative_offset; + return temp_storage.out_symbols[offset]; + } + + template <typename StateIndexT, typename SymbolIndexT> + constexpr CUDF_HOST_DEVICE OutSymbolOffsetT operator()(StateIndexT const state_id, + SymbolIndexT const match_id) const + { + return temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id + 1] - + temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id]; + } +}; + +/** + * @brief Helper class to facilitate the specification and instantiation of a DFA (i.e., the + * transition table and its number of states, the mapping of symbols to symbol groups, and the + * translation table that specifies which state transitions cause which output to be written). + * + * @tparam OutSymbolT The symbol type being output by the finite-state transducer + * @tparam NUM_SYMBOLS The number of symbol groups amongst which to differentiate (one dimension of + * the transition table) + * @tparam NUM_STATES The number of states defined by the DFA (the other dimension of the + * transition table) + */ +template <typename OutSymbolT, int32_t NUM_SYMBOLS, int32_t NUM_STATES> +class Dfa { + public: + // The maximum number of states supported by this DFA instance + // This is a value queried by the DFA simulation algorithm + static constexpr int32_t MAX_NUM_STATES = NUM_STATES; + + private: + // Symbol-group id lookup table + using SymbolGroupIdLookupT = detail::SingleSymbolSmemLUT<char>; + using SymbolGroupIdInitT = typename SymbolGroupIdLookupT::KernelParameter; + + // Transition table + using TransitionTableT = detail::TransitionTable<NUM_SYMBOLS + 1, NUM_STATES>; + using TransitionTableInitT = typename TransitionTableT::KernelParameter; + + // Translation lookup table + using OutSymbolOffsetT = uint32_t; + using TranslationTableT = detail::TransducerLookupTable<OutSymbolT, + OutSymbolOffsetT, + NUM_SYMBOLS + 1, + NUM_STATES, + (NUM_SYMBOLS + 1) * NUM_STATES>; + using TranslationTableInitT = typename TranslationTableT::KernelParameter; + + auto get_device_view() + { + return dfa_device_view<SymbolGroupIdLookupT, TransitionTableT, TranslationTableT, NUM_STATES>{ + sgid_init.d_begin(), transition_table_init.d_begin(), translation_table_init.d_begin()}; + } + + public: + template <typename StateIdT, typename SymbolGroupIdItT> + Dfa(SymbolGroupIdItT const& symbol_vec, + std::vector<std::vector<StateIdT>> const& tt_vec, + std::vector<std::vector<std::vector<OutSymbolT>>> const& out_tt_vec, + cudaStream_t stream) + { + constexpr std::size_t single_item = 1; + + sgid_init = hostdevice_vector<SymbolGroupIdInitT>{single_item, stream}; + transition_table_init = hostdevice_vector<TransitionTableInitT>{single_item, stream}; + translation_table_init = hostdevice_vector<TranslationTableInitT>{single_item, stream}; + + // Initialize symbol group id lookup table + SymbolGroupIdLookupT::InitDeviceSymbolGroupIdLut(sgid_init, symbol_vec, stream); + + // Initialize state transition table + TransitionTableT::InitDeviceTransitionTable(transition_table_init, tt_vec, stream); + + // Initialize finite-state transducer lookup table + TranslationTableT::InitDeviceTranslationTable(translation_table_init, out_tt_vec, stream); + } + + template <typename SymbolT, + typename TransducedOutItT, + typename TransducedIndexOutItT, + typename TransducedCountOutItT, + typename OffsetT> + cudaError_t Transduce(void* d_temp_storage, + size_t& temp_storage_bytes, + SymbolT const* d_chars, + OffsetT num_chars, + TransducedOutItT d_out_it, + TransducedIndexOutItT d_out_idx_it, + TransducedCountOutItT d_num_transduced_out_it, + const uint32_t seed_state = 0, + cudaStream_t stream = 0) + { + return DeviceTransduce(d_temp_storage, + temp_storage_bytes, + this->get_device_view(), + d_chars, + num_chars, + d_out_it, + d_out_idx_it, + d_num_transduced_out_it, + seed_state, + stream); + } + + private: + hostdevice_vector<SymbolGroupIdInitT> sgid_init{}; + hostdevice_vector<TransitionTableInitT> transition_table_init{}; + hostdevice_vector<TranslationTableInitT> translation_table_init{}; +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/symbol_lut.cuh b/cpp/src/io/fst/symbol_lut.cuh deleted file mode 100644 index abf71a7fbea..00000000000 --- a/cpp/src/io/fst/symbol_lut.cuh +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed 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 <cudf/types.hpp> -#include <io/utilities/hostdevice_vector.hpp> - -#include <cub/cub.cuh> - -#include <algorithm> -#include <cstdint> -#include <vector> - -namespace cudf { -namespace io { -namespace fst { -namespace detail { -/** - * @brief Class template that can be plugged into the finite-state machine to look up the symbol - * group index for a given symbol. Class template does not support multi-symbol lookups (i.e., no - * look-ahead). - * - * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id - */ -template <typename SymbolT> -class SingleSymbolSmemLUT { - private: - // Type used for representing a symbol group id (i.e., what we return for a given symbol) - using SymbolGroupIdT = uint8_t; - - /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) - static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); - - struct _TempStorage { - // sym_to_sgid[symbol] -> symbol group index - SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; - }; - - public: - struct KernelParameter { - // sym_to_sgid[min(symbol,num_valid_entries)] -> symbol group index - SymbolT num_valid_entries; - - // sym_to_sgid[symbol] -> symbol group index - SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; - }; - - using TempStorage = cub::Uninitialized<_TempStorage>; - - //------------------------------------------------------------------------------ - // HELPER METHODS - //------------------------------------------------------------------------------ - /** - * @brief - * - * @param[out] sgid_init A hostdevice_vector that will be populated - * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols - * (characters!) that correspond to the i-th symbol group index - * @param[in] stream The stream that shall be used to cudaMemcpyAsync the lookup table - * @return - */ - template <typename SymbolGroupItT> - static void InitDeviceSymbolGroupIdLut(hostdevice_vector<KernelParameter>& sgid_init, - SymbolGroupItT const& symbol_strings, - rmm::cuda_stream_view stream) - { - // The symbol group index to be returned if none of the given symbols match - SymbolGroupIdT no_match_id = symbol_strings.size(); - - // The symbol with the largest value that is mapped to a symbol group id - SymbolGroupIdT max_base_match_val = 0; - - // Initialize all entries: by default we return the no-match-id - std::fill(&sgid_init.host_ptr()->sym_to_sgid[0], - &sgid_init.host_ptr()->sym_to_sgid[NUM_ENTRIES_PER_LUT], - no_match_id); - - // Set up lookup table - uint32_t sg_id = 0; - // Iterate over the symbol groups - for (auto const& sg_symbols : symbol_strings) { - // Iterate over all symbols that belong to the current symbol group - for (auto const& sg_symbol : sg_symbols) { - max_base_match_val = std::max(max_base_match_val, static_cast<SymbolGroupIdT>(sg_symbol)); - sgid_init.host_ptr()->sym_to_sgid[static_cast<int32_t>(sg_symbol)] = sg_id; - } - sg_id++; - } - - // Initialize the out-of-bounds lookup: sym_to_sgid[max_base_match_val+1] -> no_match_id - sgid_init.host_ptr()->sym_to_sgid[max_base_match_val + 1] = no_match_id; - - // Alias memory / return memory requiremenets - // TODO I think this could be +1? - sgid_init.host_ptr()->num_valid_entries = max_base_match_val + 2; - - sgid_init.host_to_device(stream); - } - - //------------------------------------------------------------------------------ - // MEMBER VARIABLES - //------------------------------------------------------------------------------ - _TempStorage& temp_storage; - SymbolGroupIdT num_valid_entries; - - //------------------------------------------------------------------------------ - // CONSTRUCTOR - //------------------------------------------------------------------------------ - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - return private_storage; - } - - constexpr CUDF_HOST_DEVICE SingleSymbolSmemLUT(KernelParameter const& kernel_param, - TempStorage& temp_storage) - : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) - { - // GPU-side init -#if CUB_PTX_ARCH > 0 - for (int32_t i = threadIdx.x; i < kernel_param.num_valid_entries; i += blockDim.x) { - this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; - } - __syncthreads(); - -#else - // CPU-side init - for (std::size_t i = 0; i < kernel_param.num_luts; i++) { - this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; - } -#endif - } - - constexpr CUDF_HOST_DEVICE int32_t operator()(SymbolT const symbol) const - { - // Look up the symbol group for given symbol - return temp_storage.sym_to_sgid[min(symbol, num_valid_entries - 1)]; - } -}; - -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf diff --git a/cpp/src/io/fst/transition_table.cuh b/cpp/src/io/fst/transition_table.cuh deleted file mode 100644 index 5eccb926974..00000000000 --- a/cpp/src/io/fst/transition_table.cuh +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed 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 <cudf/types.hpp> -#include <cudf/utilities/error.hpp> -#include <io/utilities/hostdevice_vector.hpp> - -#include <cub/cub.cuh> - -#include <cstdint> - -namespace cudf { -namespace io { -namespace fst { -namespace detail { - -template <int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES> -class TransitionTable { - private: - // Type used - using ItemT = char; - - struct _TempStorage { - ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; - }; - - public: - using TempStorage = cub::Uninitialized<_TempStorage>; - - struct KernelParameter { - ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; - }; - - static void InitDeviceTransitionTable(hostdevice_vector<KernelParameter>& transition_table_init, - const std::vector<std::vector<int>>& trans_table, - rmm::cuda_stream_view stream) - { - // trans_table[state][symbol] -> new state - for (std::size_t state = 0; state < trans_table.size(); ++state) { - for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { - transition_table_init.host_ptr()->transitions[symbol * MAX_NUM_STATES + state] = - trans_table[state][symbol]; - } - } - - // Copy transition table to device - transition_table_init.host_to_device(stream); - } - - constexpr CUDF_HOST_DEVICE TransitionTable(const KernelParameter& kernel_param, - TempStorage& temp_storage) - : temp_storage(temp_storage.Alias()) - { -#if CUB_PTX_ARCH > 0 - for (int i = threadIdx.x; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i += blockDim.x) { - this->temp_storage.transitions[i] = kernel_param.transitions[i]; - } - __syncthreads(); -#else - for (int i = 0; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i++) { - this->temp_storage.transitions[i] = kernel_param.transitions[i]; - } -#endif - } - - /** - * @brief Returns a random-access iterator to lookup all the state transitions for one specific - * symbol from an arbitrary old_state, i.e., it[old_state] -> new_state. - * - * @param state_id The DFA's current state index from which we'll transition - * @param match_id The symbol group id of the symbol that we just read in - * @return - */ - template <typename StateIndexT, typename SymbolIndexT> - constexpr CUDF_HOST_DEVICE int32_t operator()(StateIndexT const state_id, - SymbolIndexT const match_id) const - { - return temp_storage.transitions[match_id * MAX_NUM_STATES + state_id]; - } - - private: - _TempStorage& temp_storage; - - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - - return private_storage; - } -}; - -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf diff --git a/cpp/src/io/fst/translation_table.cuh b/cpp/src/io/fst/translation_table.cuh deleted file mode 100644 index 89da994606c..00000000000 --- a/cpp/src/io/fst/translation_table.cuh +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed 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 <algorithm> -#include <cudf/types.hpp> -#include <cudf/utilities/error.hpp> -#include <io/utilities/hostdevice_vector.hpp> - -#include "rmm/device_uvector.hpp" - -#include <cub/cub.cuh> - -#include <cstdint> - -namespace cudf { -namespace io { -namespace fst { -namespace detail { - -/** - * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols - * that the finite-state transducer is supposed to output for each transition - * - * @tparam OutSymbolT The symbol type being output - * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols - * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition - * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support - * @tparam MAX_TABLE_SIZE The maximum number of items in the lookup table of output symbols - */ -template <typename OutSymbolT, - typename OutSymbolOffsetT, - int32_t MAX_NUM_SYMBOLS, - int32_t MAX_NUM_STATES, - int32_t MAX_TABLE_SIZE = (MAX_NUM_SYMBOLS * MAX_NUM_STATES)> -class TransducerLookupTable { - private: - struct _TempStorage { - OutSymbolOffsetT out_offset[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; - OutSymbolT out_symbols[MAX_TABLE_SIZE]; - }; - - public: - using TempStorage = cub::Uninitialized<_TempStorage>; - - struct KernelParameter { - OutSymbolOffsetT d_out_offsets[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; - OutSymbolT d_out_symbols[MAX_TABLE_SIZE]; - }; - - /** - * @brief Initializes the translation table (both the host and device parts) - */ - static void InitDeviceTranslationTable( - hostdevice_vector<KernelParameter>& translation_table_init, - std::vector<std::vector<std::vector<OutSymbolT>>> const& trans_table, - rmm::cuda_stream_view stream) - { - std::vector<OutSymbolT> out_symbols; - out_symbols.reserve(MAX_TABLE_SIZE); - std::vector<OutSymbolOffsetT> out_symbol_offsets; - out_symbol_offsets.reserve(MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1); - out_symbol_offsets.push_back(0); - - // Iterate over the states in the transition table - for (auto const& state_trans : trans_table) { - uint32_t num_added = 0; - // Iterate over the symbols in the transition table - for (auto const& symbol_out : state_trans) { - // Insert the output symbols for this specific (state, symbol) transition - out_symbols.insert(std::end(out_symbols), std::begin(symbol_out), std::end(symbol_out)); - out_symbol_offsets.push_back(out_symbols.size()); - num_added++; - } - - // Copy the last offset for all symbols (to guarantee a proper lookup for omitted symbols of - // this state) - if (MAX_NUM_SYMBOLS > num_added) { - int32_t count = MAX_NUM_SYMBOLS - num_added; - auto begin_it = std::prev(std::end(out_symbol_offsets)); - std::copy(begin_it, begin_it + count, std::back_inserter(out_symbol_offsets)); - } - } - - // Check whether runtime-provided table size exceeds the compile-time given max. table size - if (out_symbols.size() > MAX_TABLE_SIZE) { CUDF_FAIL("Unsupported translation table"); } - - // Prepare host-side data to be copied and passed to the device - std::copy(std::cbegin(out_symbol_offsets), - std::cend(out_symbol_offsets), - translation_table_init.host_ptr()->d_out_offsets); - std::copy(std::cbegin(out_symbols), - std::cend(out_symbols), - translation_table_init.host_ptr()->d_out_symbols); - - // Copy data to device - translation_table_init.host_to_device(stream); - } - - private: - _TempStorage& temp_storage; - - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - return private_storage; - } - - public: - /** - * @brief Synchronizes the thread block, if called from device, and, hence, requires all threads - * of the thread block to call the constructor - */ - CUDF_HOST_DEVICE TransducerLookupTable(KernelParameter const& kernel_param, - TempStorage& temp_storage) - : temp_storage(temp_storage.Alias()) - { - constexpr uint32_t num_offsets = MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1; -#if CUB_PTX_ARCH > 0 - for (int i = threadIdx.x; i < num_offsets; i += blockDim.x) { - this->temp_storage.out_offset[i] = kernel_param.d_out_offsets[i]; - } - // Make sure all threads in the block can read out_symbol_offsets[num_offsets - 1] from shared - // memory - __syncthreads(); - for (int i = threadIdx.x; i < this->temp_storage.out_offset[num_offsets - 1]; i += blockDim.x) { - this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; - } - __syncthreads(); -#else - for (int i = 0; i < num_offsets; i++) { - this->temp_storage.out_symbol_offsets[i] = kernel_param.d_out_offsets[i]; - } - for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { - this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; - } -#endif - } - - template <typename StateIndexT, typename SymbolIndexT, typename RelativeOffsetT> - constexpr CUDF_HOST_DEVICE OutSymbolT operator()(StateIndexT const state_id, - SymbolIndexT const match_id, - RelativeOffsetT const relative_offset) const - { - auto offset = temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id] + relative_offset; - return temp_storage.out_symbols[offset]; - } - - template <typename StateIndexT, typename SymbolIndexT> - constexpr CUDF_HOST_DEVICE OutSymbolOffsetT operator()(StateIndexT const state_id, - SymbolIndexT const match_id) const - { - return temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id + 1] - - temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id]; - } -}; - -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 29c93a6f3bb..012c37ab842 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -14,7 +14,7 @@ * limitations under the License. */ -#include <io/fst/device_dfa.cuh> +#include <io/fst/lookup_tables.cuh> #include <io/utilities/hostdevice_vector.hpp> #include <cudf_test/base_fixture.hpp> @@ -189,7 +189,7 @@ TEST_F(FstTest, GroundTruth) using SymbolOffsetT = uint32_t; // Helper class to set up transition table, symbol group lookup table, and translation table - using DfaFstT = cudf::io::fst::Dfa<char, (NUM_SYMBOL_GROUPS - 1), TT_NUM_STATES>; + using DfaFstT = cudf::io::fst::detail::Dfa<char, (NUM_SYMBOL_GROUPS - 1), TT_NUM_STATES>; // Prepare cuda stream for data transfers & kernels cudaStream_t stream = nullptr; From 39cff8039c160a3de0795a33c4e4fc2215072900 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 27 Apr 2022 04:42:31 -0700 Subject: [PATCH 05/81] Change interface for FST to not need temp storage --- cpp/src/io/fst/lookup_tables.cuh | 55 ++++++++++++++++++++------------ cpp/tests/io/fst/fst_test.cu | 39 ++++++++-------------- 2 files changed, 49 insertions(+), 45 deletions(-) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index 58853919b69..f8921d4091b 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -17,8 +17,8 @@ #pragma once #include <cudf/types.hpp> -#include <io/utilities/hostdevice_vector.hpp> #include <io/fst/device_dfa.cuh> +#include <io/utilities/hostdevice_vector.hpp> #include <cub/cub.cuh> @@ -485,26 +485,41 @@ class Dfa { typename TransducedIndexOutItT, typename TransducedCountOutItT, typename OffsetT> - cudaError_t Transduce(void* d_temp_storage, - size_t& temp_storage_bytes, - SymbolT const* d_chars, - OffsetT num_chars, - TransducedOutItT d_out_it, - TransducedIndexOutItT d_out_idx_it, - TransducedCountOutItT d_num_transduced_out_it, - const uint32_t seed_state = 0, - cudaStream_t stream = 0) + void Transduce(SymbolT const* d_chars, + OffsetT num_chars, + TransducedOutItT d_out_it, + TransducedIndexOutItT d_out_idx_it, + TransducedCountOutItT d_num_transduced_out_it, + const uint32_t seed_state, + rmm::cuda_stream_view stream) { - return DeviceTransduce(d_temp_storage, - temp_storage_bytes, - this->get_device_view(), - d_chars, - num_chars, - d_out_it, - d_out_idx_it, - d_num_transduced_out_it, - seed_state, - stream); + std::size_t temp_storage_bytes = 0; + rmm::device_buffer temp_storage{}; + DeviceTransduce(nullptr, + temp_storage_bytes, + this->get_device_view(), + d_chars, + num_chars, + d_out_it, + d_out_idx_it, + d_num_transduced_out_it, + seed_state, + stream); + + if (temp_storage.size() < temp_storage_bytes) { + temp_storage.resize(temp_storage_bytes, stream); + } + + DeviceTransduce(temp_storage.data(), + temp_storage_bytes, + this->get_device_view(), + d_chars, + num_chars, + d_out_it, + d_out_idx_it, + d_num_transduced_out_it, + seed_state, + stream); } private: diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 012c37ab842..9e8011bb7df 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -217,8 +217,10 @@ TEST_F(FstTest, GroundTruth) input += input; // Prepare input & output buffers + constexpr std::size_t single_item = 1; rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); hostdevice_vector<SymbolT> output_gpu(input.size(), stream_view); + hostdevice_vector<SymbolOffsetT> output_gpu_size(single_item, stream_view); hostdevice_vector<SymbolOffsetT> out_indexes_gpu(input.size(), stream_view); ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); @@ -228,32 +230,19 @@ TEST_F(FstTest, GroundTruth) std::size_t temp_storage_bytes = 0; - // Query temporary storage requirements - ASSERT_CUDA_SUCCEEDED(parser.Transduce(nullptr, - temp_storage_bytes, - d_input.data(), - static_cast<SymbolOffsetT>(d_input.size()), - output_gpu.device_ptr(), - out_indexes_gpu.device_ptr(), - cub::DiscardOutputIterator<int32_t>{}, - start_state, - stream)); - // Allocate device-side temporary storage & run algorithm - rmm::device_buffer temp_storage{temp_storage_bytes, stream_view}; - ASSERT_CUDA_SUCCEEDED(parser.Transduce(temp_storage.data(), - temp_storage_bytes, - d_input.data(), - static_cast<SymbolOffsetT>(d_input.size()), - output_gpu.device_ptr(), - out_indexes_gpu.device_ptr(), - cub::DiscardOutputIterator<int32_t>{}, - start_state, - stream)); + parser.Transduce(d_input.data(), + static_cast<SymbolOffsetT>(d_input.size()), + output_gpu.device_ptr(), + out_indexes_gpu.device_ptr(), + output_gpu_size.device_ptr(), + start_state, + stream); // Async copy results from device to host output_gpu.device_to_host(stream_view); out_indexes_gpu.device_to_host(stream_view); + output_gpu_size.device_to_host(stream_view); // Prepare CPU-side results for verification std::string output_cpu{}; @@ -275,13 +264,13 @@ TEST_F(FstTest, GroundTruth) cudaStreamSynchronize(stream); // Verify results - ASSERT_EQ(output_gpu.size(), output_cpu.size()); + ASSERT_EQ(output_gpu_size[0], output_cpu.size()); ASSERT_EQ(out_indexes_gpu.size(), out_index_cpu.size()); - for (std::size_t i = 0; i < output_gpu.size(); i++) { - ASSERT_EQ(output_gpu.host_ptr()[i], output_cpu[i]) << "Mismatch at index #" << i; + for (std::size_t i = 0; i < output_cpu.size(); i++) { + ASSERT_EQ(output_gpu[i], output_cpu[i]) << "Mismatch at index #" << i; } for (std::size_t i = 0; i < out_indexes_gpu.size(); i++) { - ASSERT_EQ(out_indexes_gpu.host_ptr()[i], out_index_cpu[i]) << "Mismatch at index #" << i; + ASSERT_EQ(out_indexes_gpu[i], out_index_cpu[i]) << "Mismatch at index #" << i; } } From e24a13301a34fbb08d8424a2ba4edfbebd402d67 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 4 May 2022 07:29:00 -0700 Subject: [PATCH 06/81] removing unused var post-cleanup --- cpp/tests/io/fst/fst_test.cu | 2 -- 1 file changed, 2 deletions(-) diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 9e8011bb7df..3d4f68b03c4 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -228,8 +228,6 @@ TEST_F(FstTest, GroundTruth) // Run algorithm DfaFstT parser{pda_sgs, pda_state_tt, pda_out_tt, stream}; - std::size_t temp_storage_bytes = 0; - // Allocate device-side temporary storage & run algorithm parser.Transduce(d_input.data(), static_cast<SymbolOffsetT>(d_input.size()), From caf61955c32c57cca287fb9d7e74bf5d0efc8506 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 9 May 2022 10:24:51 -0700 Subject: [PATCH 07/81] unified usage of pragma unrolls --- cpp/src/io/fst/in_reg_array.cuh | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index ed5948249d4..3180dbfe132 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -59,7 +59,7 @@ class MultiFragmentInRegArray { //------------------------------------------------------------------------------ // HELPER FUNCTIONS //------------------------------------------------------------------------------ - __device__ __host__ __forceinline__ uint32_t bfe(const uint32_t& data, + __host__ __device__ __forceinline__ uint32_t bfe(const uint32_t& data, uint32_t bit_start, uint32_t num_bits) const { @@ -71,7 +71,7 @@ class MultiFragmentInRegArray { #endif } - __device__ __host__ __forceinline__ void bfi(uint32_t& data, + __host__ __device__ __forceinline__ void bfi(uint32_t& data, uint32_t bits, uint32_t bit_start, uint32_t num_bits) const @@ -97,7 +97,6 @@ class MultiFragmentInRegArray { { uint32_t val = 0; - // #pragma unroll for (uint32_t i = 0; i < FRAGMENTS_PER_ITEM; ++i) { val = val | bfe(data[i], index * BITS_PER_FRAG_ITEM, BITS_PER_FRAG_ITEM) << (i * BITS_PER_FRAG_ITEM); @@ -107,7 +106,6 @@ class MultiFragmentInRegArray { __host__ __device__ __forceinline__ void Set(uint32_t index, uint32_t value) { - // #pragma unroll for (uint32_t i = 0; i < FRAGMENTS_PER_ITEM; ++i) { uint32_t frag_bits = bfe(value, i * BITS_PER_FRAG_ITEM, BITS_PER_FRAG_ITEM); bfi(data[i], frag_bits, index * BITS_PER_FRAG_ITEM, BITS_PER_FRAG_ITEM); From ea79a81fb9b0473d37f31c42bce25269a3d17d88 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 9 May 2022 10:32:17 -0700 Subject: [PATCH 08/81] Adding hostdevice macros to in-reg array --- cpp/src/io/fst/in_reg_array.cuh | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index 3180dbfe132..1180dc594da 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -15,6 +15,8 @@ */ #pragma once +#include <cudf/types.hpp> + #include <cub/cub.cuh> #include <cstdint> @@ -59,7 +61,7 @@ class MultiFragmentInRegArray { //------------------------------------------------------------------------------ // HELPER FUNCTIONS //------------------------------------------------------------------------------ - __host__ __device__ __forceinline__ uint32_t bfe(const uint32_t& data, + CUDF_HOST_DEVICE uint32_t bfe(const uint32_t& data, uint32_t bit_start, uint32_t num_bits) const { @@ -71,7 +73,7 @@ class MultiFragmentInRegArray { #endif } - __host__ __device__ __forceinline__ void bfi(uint32_t& data, + CUDF_HOST_DEVICE void bfi(uint32_t& data, uint32_t bits, uint32_t bit_start, uint32_t num_bits) const @@ -93,7 +95,7 @@ class MultiFragmentInRegArray { // ACCESSORS //------------------------------------------------------------------------------ public: - __host__ __device__ __forceinline__ uint32_t Get(int32_t index) const + CUDF_HOST_DEVICE uint32_t Get(int32_t index) const { uint32_t val = 0; @@ -104,7 +106,7 @@ class MultiFragmentInRegArray { return val; } - __host__ __device__ __forceinline__ void Set(uint32_t index, uint32_t value) + CUDF_HOST_DEVICE void Set(uint32_t index, uint32_t value) { for (uint32_t i = 0; i < FRAGMENTS_PER_ITEM; ++i) { uint32_t frag_bits = bfe(value, i * BITS_PER_FRAG_ITEM, BITS_PER_FRAG_ITEM); @@ -115,14 +117,14 @@ class MultiFragmentInRegArray { //------------------------------------------------------------------------------ // CONSTRUCTORS //------------------------------------------------------------------------------ - __host__ __device__ __forceinline__ MultiFragmentInRegArray() + CUDF_HOST_DEVICE MultiFragmentInRegArray() { for (uint32_t i = 0; i < FRAGMENTS_PER_ITEM; ++i) { data[i] = 0; } } - __host__ __device__ __forceinline__ MultiFragmentInRegArray(uint32_t const (&array)[NUM_ITEMS]) + CUDF_HOST_DEVICE MultiFragmentInRegArray(uint32_t const (&array)[NUM_ITEMS]) { for (uint32_t i = 0; i < NUM_ITEMS; ++i) { Set(i, array[i]); From 17dcbfd07b73a64a0a1cfda71ea9c2770b6a8662 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 9 May 2022 10:33:00 -0700 Subject: [PATCH 09/81] making const vars const --- cpp/src/io/fst/agent_dfa.cuh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index 3bc59160696..aaafd2d7a22 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -104,7 +104,7 @@ class DFASimulationCallbackWrapper { StateVectorT const& new_state, SymbolIndexT const& symbol_id) { - uint32_t count = transducer_table(old_state.Get(0), symbol_id); + uint32_t const count = transducer_table(old_state.Get(0), symbol_id); if (write) { for (uint32_t out_char = 0; out_char < count; out_char++) { out_it[out_count + out_char] = transducer_table(old_state.Get(0), symbol_id, out_char); @@ -117,7 +117,7 @@ class DFASimulationCallbackWrapper { __host__ __device__ __forceinline__ void TearDown() {} public: - TransducerTableT transducer_table; + TransducerTableT const transducer_table; TransducedOutItT out_it; TransducedIndexOutItT out_idx_it; uint32_t out_count; From 6fdd24a5625150469242af16fdcb1d549b3676e0 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 9 May 2022 12:17:34 -0700 Subject: [PATCH 10/81] refactor lut sanity check --- cpp/src/io/fst/lookup_tables.cuh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index f8921d4091b..3e5504a6208 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -343,7 +343,7 @@ class TransducerLookupTable { } // Check whether runtime-provided table size exceeds the compile-time given max. table size - if (out_symbols.size() > MAX_TABLE_SIZE) { CUDF_FAIL("Unsupported translation table"); } + CUDF_EXPECTS(out_symbols.size() <= MAX_TABLE_SIZE, "Unsupported translation table"); // Prepare host-side data to be copied and passed to the device std::copy(std::cbegin(out_symbol_offsets), From eccf9701432f557b52b1f44b985128668bf1462f Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 2 Jun 2022 05:19:53 -0700 Subject: [PATCH 11/81] fixes sg-count & uses rmm stream in fst tests --- cpp/src/io/fst/lookup_tables.cuh | 10 +++++----- cpp/tests/io/fst/fst_test.cu | 31 ++++++++++++++++--------------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index 3e5504a6208..f245aa76368 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -421,8 +421,8 @@ class TransducerLookupTable { * translation table that specifies which state transitions cause which output to be written). * * @tparam OutSymbolT The symbol type being output by the finite-state transducer - * @tparam NUM_SYMBOLS The number of symbol groups amongst which to differentiate (one dimension of - * the transition table) + * @tparam NUM_SYMBOLS The number of symbol groups amongst which to differentiate including the + * wildcard symbol group (one dimension of the transition table) * @tparam NUM_STATES The number of states defined by the DFA (the other dimension of the * transition table) */ @@ -439,16 +439,16 @@ class Dfa { using SymbolGroupIdInitT = typename SymbolGroupIdLookupT::KernelParameter; // Transition table - using TransitionTableT = detail::TransitionTable<NUM_SYMBOLS + 1, NUM_STATES>; + using TransitionTableT = detail::TransitionTable<NUM_SYMBOLS, NUM_STATES>; using TransitionTableInitT = typename TransitionTableT::KernelParameter; // Translation lookup table using OutSymbolOffsetT = uint32_t; using TranslationTableT = detail::TransducerLookupTable<OutSymbolT, OutSymbolOffsetT, - NUM_SYMBOLS + 1, + NUM_SYMBOLS, NUM_STATES, - (NUM_SYMBOLS + 1) * NUM_STATES>; + NUM_SYMBOLS * NUM_STATES>; using TranslationTableInitT = typename TranslationTableT::KernelParameter; auto get_device_view() diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 3d4f68b03c4..8c43aa92455 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -23,6 +23,7 @@ #include <cudf/types.hpp> #include <rmm/cuda_stream_view.hpp> +#include <rmm/cuda_stream.hpp> #include <rmm/device_buffer.hpp> #include <rmm/device_uvector.hpp> @@ -189,12 +190,10 @@ TEST_F(FstTest, GroundTruth) using SymbolOffsetT = uint32_t; // Helper class to set up transition table, symbol group lookup table, and translation table - using DfaFstT = cudf::io::fst::detail::Dfa<char, (NUM_SYMBOL_GROUPS - 1), TT_NUM_STATES>; + using DfaFstT = cudf::io::fst::detail::Dfa<char, NUM_SYMBOL_GROUPS, TT_NUM_STATES>; // Prepare cuda stream for data transfers & kernels - cudaStream_t stream = nullptr; - cudaStreamCreate(&stream); - rmm::cuda_stream_view stream_view(stream); + rmm::cuda_stream stream{}; // Test input std::string input = R"( {)" @@ -216,17 +215,19 @@ TEST_F(FstTest, GroundTruth) for (std::size_t i = 0; i < 10; i++) input += input; + + // Prepare input & output buffers constexpr std::size_t single_item = 1; - rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); - hostdevice_vector<SymbolT> output_gpu(input.size(), stream_view); - hostdevice_vector<SymbolOffsetT> output_gpu_size(single_item, stream_view); - hostdevice_vector<SymbolOffsetT> out_indexes_gpu(input.size(), stream_view); + rmm::device_uvector<SymbolT> d_input(input.size(), stream.view()); + hostdevice_vector<SymbolT> output_gpu(input.size(), stream.view()); + hostdevice_vector<SymbolOffsetT> output_gpu_size(single_item, stream.view()); + hostdevice_vector<SymbolOffsetT> out_indexes_gpu(input.size(), stream.view()); ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( - d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); + d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream.value())); // Run algorithm - DfaFstT parser{pda_sgs, pda_state_tt, pda_out_tt, stream}; + DfaFstT parser{pda_sgs, pda_state_tt, pda_out_tt, stream.value()}; // Allocate device-side temporary storage & run algorithm parser.Transduce(d_input.data(), @@ -235,12 +236,12 @@ TEST_F(FstTest, GroundTruth) out_indexes_gpu.device_ptr(), output_gpu_size.device_ptr(), start_state, - stream); + stream.value()); // Async copy results from device to host - output_gpu.device_to_host(stream_view); - out_indexes_gpu.device_to_host(stream_view); - output_gpu_size.device_to_host(stream_view); + output_gpu.device_to_host(stream.view()); + out_indexes_gpu.device_to_host(stream.view()); + output_gpu_size.device_to_host(stream.view()); // Prepare CPU-side results for verification std::string output_cpu{}; @@ -259,7 +260,7 @@ TEST_F(FstTest, GroundTruth) std::back_inserter(out_index_cpu)); // Make sure results have been copied back to host - cudaStreamSynchronize(stream); + cudaStreamSynchronize(stream.value()); // Verify results ASSERT_EQ(output_gpu_size[0], output_cpu.size()); From 9fe8e4b6e2c527e471d9627369e72595ef3e452c Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Tue, 14 Jun 2022 03:12:35 -0700 Subject: [PATCH 12/81] minor doxygen fix --- cpp/src/io/fst/device_dfa.cuh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/fst/device_dfa.cuh b/cpp/src/io/fst/device_dfa.cuh index d3f0e8be213..10c3b9ac814 100644 --- a/cpp/src/io/fst/device_dfa.cuh +++ b/cpp/src/io/fst/device_dfa.cuh @@ -29,8 +29,8 @@ namespace fst { * @brief Uses a deterministic finite automaton to transduce a sequence of symbols from an input * iterator to a sequence of transduced output symbols. * - * @tparam SymbolItT Random-access input iterator type to symbols fed into the FST * @tparam DfaT The DFA specification + * @tparam SymbolItT Random-access input iterator type to symbols fed into the FST * @tparam TransducedOutItT Random-access output iterator to which the transduced output will be * written * @tparam TransducedIndexOutItT Random-access output iterator type to which the indexes of the From 694a365448a2156d0a1c60fafcd52f67c5f0c3f6 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 15 Jun 2022 04:28:51 -0700 Subject: [PATCH 13/81] adopts suggested fst test changes --- cpp/tests/io/fst/fst_test.cu | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 8c43aa92455..b6d5327bb59 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -20,15 +20,16 @@ #include <cudf_test/base_fixture.hpp> #include <cudf_test/cudf_gtest.hpp> +#include <cudf/scalar/scalar_factories.hpp> +#include <cudf/strings/repeat_strings.hpp> #include <cudf/types.hpp> +#include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> #include <rmm/cuda_stream.hpp> #include <rmm/device_buffer.hpp> #include <rmm/device_uvector.hpp> -#include "cub/cub.cuh" - #include <cstdlib> #include <vector> @@ -212,8 +213,13 @@ TEST_F(FstTest, GroundTruth) R"(} {} [] [ ])"; // Repeat input sample 1024x - for (std::size_t i = 0; i < 10; i++) - input += input; + size_t string_size = 1 << 10; + auto d_input_scalar = cudf::make_string_scalar(input); + auto& d_string_scalar = static_cast<cudf::string_scalar&>(*d_input_scalar); + const cudf::size_type repeat_times = string_size / input.size(); + auto d_input_string = cudf::strings::repeat_string(d_string_scalar, repeat_times); + auto& d_input = static_cast<cudf::scalar_type_t<std::string>&>(*d_input_string); + input = d_input.to_string(stream); @@ -260,7 +266,7 @@ TEST_F(FstTest, GroundTruth) std::back_inserter(out_index_cpu)); // Make sure results have been copied back to host - cudaStreamSynchronize(stream.value()); + stream.synchronize(); // Verify results ASSERT_EQ(output_gpu_size[0], output_cpu.size()); From f656f494e39f628dee12706ee2a0e9c6ea180126 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 7 Jul 2022 02:41:16 -0700 Subject: [PATCH 14/81] adopts device-side test data gen --- cpp/tests/io/fst/fst_test.cu | 34 +++++++++------------------------- 1 file changed, 9 insertions(+), 25 deletions(-) diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index b6d5327bb59..0c337be61f3 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -26,7 +26,6 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> -#include <rmm/cuda_stream.hpp> #include <rmm/device_buffer.hpp> #include <rmm/device_uvector.hpp> @@ -91,28 +90,18 @@ static std::pair<OutputItT, IndexOutputItT> fst_baseline(InputItT begin, auto const& symbol = *it; std::size_t symbol_group = 0; - bool found = false; // Iterate over symbol groups and search for the first symbol group containing the current // symbol for (auto const& sg : symbol_group_lut) { - for (auto const& s : sg) - if (s == symbol) found = true; - if (found) break; + if (std::find(std::cbegin(sg), std::cend(sg), symbol) != std::cend(sg)) { break; } symbol_group++; } // Output the translated symbols to the output tape - size_t inserted = 0; for (auto out : translation_table[state][symbol_group]) { - // std::cout << in_offset << ": " << out << "\n"; *out_tape = out; ++out_tape; - inserted++; - } - - // Output the index of the current symbol, iff it caused some output to be written - if (inserted > 0) { *out_index_tape = in_offset; out_index_tape++; } @@ -120,6 +109,7 @@ static std::pair<OutputItT, IndexOutputItT> fst_baseline(InputItT begin, // Transition the state of the finite-state machine state = transition_table[state][symbol_group]; + // Continue with next symbol from input tape in_offset++; } return {out_tape, out_index_tape}; @@ -195,10 +185,11 @@ TEST_F(FstTest, GroundTruth) // Prepare cuda stream for data transfers & kernels rmm::cuda_stream stream{}; + rmm::cuda_stream_view stream_view(stream); // Test input std::string input = R"( {)" - R"(category": "reference",)" + R"("category": "reference",)" R"("index:" [4,12,42],)" R"("author": "Nigel Rees",)" R"("title": "Sayings of the Century",)" @@ -212,8 +203,7 @@ TEST_F(FstTest, GroundTruth) R"("price": 8.95)" R"(} {} [] [ ])"; - // Repeat input sample 1024x - size_t string_size = 1 << 10; + size_t string_size = input.size() * (1 << 10); auto d_input_scalar = cudf::make_string_scalar(input); auto& d_string_scalar = static_cast<cudf::string_scalar&>(*d_input_scalar); const cudf::size_type repeat_times = string_size / input.size(); @@ -221,16 +211,11 @@ TEST_F(FstTest, GroundTruth) auto& d_input = static_cast<cudf::scalar_type_t<std::string>&>(*d_input_string); input = d_input.to_string(stream); - - // Prepare input & output buffers constexpr std::size_t single_item = 1; - rmm::device_uvector<SymbolT> d_input(input.size(), stream.view()); - hostdevice_vector<SymbolT> output_gpu(input.size(), stream.view()); - hostdevice_vector<SymbolOffsetT> output_gpu_size(single_item, stream.view()); - hostdevice_vector<SymbolOffsetT> out_indexes_gpu(input.size(), stream.view()); - ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( - d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream.value())); + hostdevice_vector<SymbolT> output_gpu(input.size(), stream_view); + hostdevice_vector<SymbolOffsetT> output_gpu_size(single_item, stream_view); + hostdevice_vector<SymbolOffsetT> out_indexes_gpu(input.size(), stream_view); // Run algorithm DfaFstT parser{pda_sgs, pda_state_tt, pda_out_tt, stream.value()}; @@ -270,11 +255,10 @@ TEST_F(FstTest, GroundTruth) // Verify results ASSERT_EQ(output_gpu_size[0], output_cpu.size()); - ASSERT_EQ(out_indexes_gpu.size(), out_index_cpu.size()); for (std::size_t i = 0; i < output_cpu.size(); i++) { ASSERT_EQ(output_gpu[i], output_cpu[i]) << "Mismatch at index #" << i; } - for (std::size_t i = 0; i < out_indexes_gpu.size(); i++) { + for (std::size_t i = 0; i < output_cpu.size(); i++) { ASSERT_EQ(out_indexes_gpu[i], out_index_cpu[i]) << "Mismatch at index #" << i; } } From 485a1c632bc6fe03030c445b8cff06dc1c3ca32f Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Fri, 8 Jul 2022 22:49:57 -0700 Subject: [PATCH 15/81] adopts c++17 namespaces declarations --- cpp/src/io/fst/agent_dfa.cuh | 10 ++-------- cpp/src/io/fst/device_dfa.cuh | 8 ++------ cpp/src/io/fst/dispatch_dfa.cuh | 10 ++-------- cpp/src/io/fst/in_reg_array.cuh | 20 ++++++-------------- cpp/src/io/fst/lookup_tables.cuh | 10 ++-------- 5 files changed, 14 insertions(+), 44 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index aaafd2d7a22..f641cd3e053 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -19,10 +19,7 @@ #include <cub/cub.cuh> -namespace cudf { -namespace io { -namespace fst { -namespace detail { +namespace cudf::io::fst::detail { //----------------------------------------------------------------------------- // STATE VECTOR @@ -715,7 +712,4 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ } } -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf +} // namespace cudf::io::fst::detail diff --git a/cpp/src/io/fst/device_dfa.cuh b/cpp/src/io/fst/device_dfa.cuh index 10c3b9ac814..56afc32e643 100644 --- a/cpp/src/io/fst/device_dfa.cuh +++ b/cpp/src/io/fst/device_dfa.cuh @@ -21,9 +21,7 @@ #include <cstdint> -namespace cudf { -namespace io { -namespace fst { +namespace cudf::io::fst { /** * @brief Uses a deterministic finite automaton to transduce a sequence of symbols from an input @@ -93,6 +91,4 @@ cudaError_t DeviceTransduce(void* d_temp_storage, stream); } -} // namespace fst -} // namespace io -} // namespace cudf +} // namespace cudf::io::fst diff --git a/cpp/src/io/fst/dispatch_dfa.cuh b/cpp/src/io/fst/dispatch_dfa.cuh index 316d6ea0d5f..6de1757bfed 100644 --- a/cpp/src/io/fst/dispatch_dfa.cuh +++ b/cpp/src/io/fst/dispatch_dfa.cuh @@ -22,10 +22,7 @@ #include <cstdint> -namespace cudf { -namespace io { -namespace fst { -namespace detail { +namespace cudf::io::fst::detail { /** * @brief The tuning policy comprising all the architecture-specific compile-time tuning parameters. @@ -456,7 +453,4 @@ struct DispatchFSM : DeviceFSMPolicy { sm_count, stv_tile_state, fst_offset_tile_state, seed_state, d_thread_state_transition); } }; -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf +} // namespace cudf::io::fst::detail diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index 1180dc594da..9d59d04cb9b 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -21,10 +21,7 @@ #include <cstdint> -namespace cudf { -namespace io { -namespace fst { -namespace detail { +namespace cudf::io::fst::detail { /** * @brief A bit-packed array of items that can be backed by registers yet allows to be dynamically @@ -61,9 +58,7 @@ class MultiFragmentInRegArray { //------------------------------------------------------------------------------ // HELPER FUNCTIONS //------------------------------------------------------------------------------ - CUDF_HOST_DEVICE uint32_t bfe(const uint32_t& data, - uint32_t bit_start, - uint32_t num_bits) const + CUDF_HOST_DEVICE uint32_t bfe(const uint32_t& data, uint32_t bit_start, uint32_t num_bits) const { #if CUB_PTX_ARCH > 0 return cub::BFE(data, bit_start, num_bits); @@ -74,9 +69,9 @@ class MultiFragmentInRegArray { } CUDF_HOST_DEVICE void bfi(uint32_t& data, - uint32_t bits, - uint32_t bit_start, - uint32_t num_bits) const + uint32_t bits, + uint32_t bit_start, + uint32_t num_bits) const { #if CUB_PTX_ARCH > 0 cub::BFI(data, data, bits, bit_start, num_bits); @@ -132,7 +127,4 @@ class MultiFragmentInRegArray { } }; -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf +} // namespace cudf::io::fst::detail diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index f245aa76368..f1dd31d1f4b 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -26,10 +26,7 @@ #include <cstdint> #include <vector> -namespace cudf { -namespace io { -namespace fst { -namespace detail { +namespace cudf::io::fst::detail { /** * @brief Class template that can be plugged into the finite-state machine to look up the symbol @@ -528,7 +525,4 @@ class Dfa { hostdevice_vector<TranslationTableInitT> translation_table_init{}; }; -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf +} // namespace cudf::io::fst::detail From 5f1c4b544882f1d35ac8701eb611e8f64c12ac56 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 11 Jul 2022 06:26:47 -0700 Subject: [PATCH 16/81] removes state vector-wrapper in favor of vanilla array --- cpp/src/io/fst/agent_dfa.cuh | 152 ++++++++++++----------------------- 1 file changed, 51 insertions(+), 101 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index f641cd3e053..65918a33e5e 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -21,35 +21,6 @@ namespace cudf::io::fst::detail { -//----------------------------------------------------------------------------- -// STATE VECTOR -//----------------------------------------------------------------------------- -/** - * @brief A vector is able to hold multiple state indices (e.g., to represent multiple DFA - * instances, where the i-th item would represent the i-th DFA instance). - * - * @tparam StateIndexT Signed or unsigned type used to index items inside the vector - * @tparam NUM_ITEMS The number of items to be allocated for a vector - */ -template <typename StateIndexT, int32_t NUM_ITEMS> -class MultiItemStateVector { - public: - template <typename IndexT> - __host__ __device__ __forceinline__ void Set(IndexT index, StateIndexT value) noexcept - { - state_[index] = value; - } - - template <typename IndexT> - __host__ __device__ __forceinline__ StateIndexT Get(IndexT index) const noexcept - { - return state_[index]; - } - - private: - StateIndexT state_[NUM_ITEMS]; -}; - //----------------------------------------------------------------------------- // DFA-SIMULATION STATE COMPOSITION FUNCTORS //----------------------------------------------------------------------------- @@ -57,10 +28,15 @@ class MultiItemStateVector { * @brief Implements an associative composition operation for state transition vectors and * offset-to-overap vectors to be used with a prefix scan. * + * Read the following table as follows: c = op(l,r), where op is the composition operator. + * For row 0: l maps 0 to 2. r maps 2 to 2. Hence, the result for 0 is 2. + * For row 1: l maps 1 to 1. r maps 1 to 2. Hence, the result for 1 is 2. + * For row 2: l maps 2 to 0. r maps 0 to 1. Hence, the result for 2 is 1. + * * l r = c ( s->l->r) * 0: [2] [1] [2] (i.e. 0->2->2) * 1: [1] [2] [2] (i.e. 1->1->2) - * 2: [0] [2] [1] (i.e. 2->0->2) + * 2: [0] [2] [1] (i.e. 2->0->1) * @tparam NUM_ITEMS The number of items stored within a vector */ template <int32_t NUM_ITEMS> @@ -68,7 +44,7 @@ struct VectorCompositeOp { template <typename VectorT> __host__ __device__ __forceinline__ VectorT operator()(VectorT const& lhs, VectorT const& rhs) { - VectorT res; + VectorT res{}; for (int32_t i = 0; i < NUM_ITEMS; ++i) { res.Set(i, rhs.Get(lhs.Get(i))); } @@ -95,16 +71,16 @@ class DFASimulationCallbackWrapper { if (!write) out_count = 0; } - template <typename CharIndexT, typename StateVectorT, typename SymbolIndexT> - __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const& character_index, - StateVectorT const& old_state, - StateVectorT const& new_state, - SymbolIndexT const& symbol_id) + template <typename CharIndexT, typename StateIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const character_index, + StateIndexT const old_state, + StateIndexT const new_state, + SymbolIndexT const symbol_id) { - uint32_t const count = transducer_table(old_state.Get(0), symbol_id); + uint32_t const count = transducer_table(old_state, symbol_id); if (write) { for (uint32_t out_char = 0; out_char < count; out_char++) { - out_it[out_count + out_char] = transducer_table(old_state.Get(0), symbol_id, out_char); + out_it[out_count + out_char] = transducer_table(old_state, symbol_id, out_char); out_idx_it[out_count + out_char] = offset + character_index; } } @@ -125,22 +101,11 @@ class DFASimulationCallbackWrapper { //----------------------------------------------------------------------------- // STATE-TRANSITION CALLBACKS //----------------------------------------------------------------------------- -class StateTransitionCallbackOp { +template <int32_t NUM_INSTANCES, typename TransitionTableT> +class StateVectorTransitionOp { public: - template <typename CharIndexT, typename SymbolIndexT> - __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const& character_index, - SymbolIndexT const& read_symbol_id) const - { - } -}; -/// Type alias for a state transition callback class that performs no operation on any callback -using NoOpStateTransitionOp = StateTransitionCallbackOp; - -template <int32_t NUM_INSTANCES, typename StateVectorT, typename TransitionTableT> -class StateVectorTransitionOp : public StateTransitionCallbackOp { - public: - __host__ __device__ __forceinline__ - StateVectorTransitionOp(TransitionTableT const& transition_table, StateVectorT& state_vector) + __host__ __device__ __forceinline__ StateVectorTransitionOp( + TransitionTableT const& transition_table, std::array<int32_t, NUM_INSTANCES>& state_vector) : transition_table(transition_table), state_vector(state_vector) { } @@ -150,39 +115,37 @@ class StateVectorTransitionOp : public StateTransitionCallbackOp { SymbolIndexT const read_symbol_id) const { for (int32_t i = 0; i < NUM_INSTANCES; ++i) { - state_vector.Set(i, transition_table(state_vector.Get(i), read_symbol_id)); + state_vector[i] = transition_table(state_vector[i], read_symbol_id); } } public: - StateVectorT& state_vector; + std::array<int32_t, NUM_INSTANCES>& state_vector; const TransitionTableT& transition_table; }; -template <typename CallbackOpT, typename StateVectorT, typename TransitionTableT> +template <typename CallbackOpT, typename TransitionTableT> struct StateTransitionOp { - StateVectorT old_state_vector; - StateVectorT state_vector; + int32_t state; const TransitionTableT& transition_table; CallbackOpT& callback_op; - __host__ __device__ __forceinline__ StateTransitionOp(const TransitionTableT& transition_table, - StateVectorT state_vector, + __host__ __device__ __forceinline__ StateTransitionOp(TransitionTableT const& transition_table, + int32_t state, CallbackOpT& callback_op) - : transition_table(transition_table), - state_vector(state_vector), - old_state_vector(state_vector), - callback_op(callback_op) + : transition_table(transition_table), state(state), callback_op(callback_op) { } template <typename CharIndexT, typename SymbolIndexT> - __host__ __device__ __forceinline__ void ReadSymbol(const CharIndexT& character_index, - const SymbolIndexT& read_symbol_id) + __host__ __device__ __forceinline__ void ReadSymbol(CharIndexT const& character_index, + SymbolIndexT const& read_symbol_id) { - old_state_vector = state_vector; - state_vector.Set(0, transition_table(state_vector.Get(0), read_symbol_id)); - callback_op.ReadSymbol(character_index, old_state_vector, state_vector, read_symbol_id); + // Remember what state we were in before we made the transition + int32_t previous_state = state; + + state = transition_table(state, read_symbol_id); + callback_op.ReadSymbol(character_index, previous_state, state, read_symbol_id); } }; @@ -237,9 +200,6 @@ struct AgentDFA { { } - //--------------------------------------------------------------------- - // STATIC PARSING PRIMITIVES - //--------------------------------------------------------------------- template <int32_t NUM_SYMBOLS, // The net (excluding overlap) number of characters to be parsed typename SymbolMatcherT, // The symbol matcher returning the matched symbol and its // length @@ -251,13 +211,11 @@ struct AgentDFA { CallbackOpT callback_op, cub::Int2Type<IS_FULL_BLOCK> /*IS_FULL_BLOCK*/) { - uint32_t matched_id; - // Iterate over symbols #pragma unroll for (int32_t i = 0; i < NUM_SYMBOLS; ++i) { if (IS_FULL_BLOCK || threadIdx.x * SYMBOLS_PER_THREAD + i < max_num_chars) { - matched_id = symbol_matcher(chars[i]); + uint32_t matched_id = symbol_matcher(chars[i]); callback_op.ReadSymbol(i, matched_id); } } @@ -400,20 +358,16 @@ struct AgentDFA { } } - template <int32_t NUM_STATES, - typename SymbolMatcherT, - typename TransitionTableT, - typename StateVectorT> + template <int32_t NUM_STATES, typename SymbolMatcherT, typename TransitionTableT> __device__ __forceinline__ void GetThreadStateTransitionVector( const SymbolMatcherT& symbol_matcher, const TransitionTableT& transition_table, const CharT* d_chars, const OffsetT block_offset, const OffsetT num_total_symbols, - StateVectorT& state_vector) + std::array<int32_t, NUM_STATES>& state_vector) { - using StateVectorTransitionOpT = - StateVectorTransitionOp<NUM_STATES, StateVectorT, TransitionTableT>; + using StateVectorTransitionOpT = StateVectorTransitionOp<NUM_STATES, TransitionTableT>; // Start parsing and to transition states StateVectorTransitionOpT transition_op(transition_table, state_vector); @@ -439,14 +393,11 @@ struct AgentDFA { GetThreadStateTransitions<SYMBOLS_PER_THREAD>( symbol_matcher, t_chars, num_block_chars, transition_op, cub::Int2Type<false>()); } - - // transition_op.TearDown(); } template <int32_t BYPASS_LOAD, typename SymbolMatcherT, typename TransitionTableT, - typename StateVectorT, typename CallbackOpT> __device__ __forceinline__ void GetThreadStateTransitions( SymbolMatcherT const& symbol_matcher, @@ -454,14 +405,14 @@ struct AgentDFA { CharT const* d_chars, OffsetT const block_offset, OffsetT const num_total_symbols, - StateVectorT& state_vector, + int32_t& state, CallbackOpT& callback_op, cub::Int2Type<BYPASS_LOAD> /**/) { - using StateTransitionOpT = StateTransitionOp<CallbackOpT, StateVectorT, TransitionTableT>; + using StateTransitionOpT = StateTransitionOp<CallbackOpT, TransitionTableT>; // Start parsing and to transition states - StateTransitionOpT transition_op(transition_table, state_vector, callback_op); + StateTransitionOpT transition_op(transition_table, state, callback_op); // Load characters into shared memory if (!BYPASS_LOAD) LoadBlock(d_chars, block_offset, num_total_symbols); @@ -528,7 +479,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ SYMBOLS_PER_BLOCK = AgentDfaSimT::SYMBOLS_PER_BLOCK }; - // Shared memory required by the DFA simulator + // Shared memory required by the DFA simulation algorithm __shared__ typename AgentDfaSimT::TempStorage dfa_storage; // Shared memory required by the symbol group lookup table @@ -552,18 +503,18 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ // Set up DFA AgentDfaSimT agent_dfa(dfa_storage); - // Memory is the state transition vector passed on to the second stage of the algorithm + // The state transition vector passed on to the second stage of the algorithm StateVectorT out_state_vector; // Stage 1: Compute the state-transition vector if (IS_TRANS_VECTOR_PASS || IS_SINGLE_PASS) { - // StateVectorT state_vector; - MultiItemStateVector<int32_t, NUM_STATES> state_vector; + // Keeping track of the state for each of the <NUM_STATES> state machines + std::array<int32_t, NUM_STATES> state_vector; // Initialize the seed state transition vector with the identity vector #pragma unroll for (int32_t i = 0; i < NUM_STATES; ++i) { - state_vector.Set(i, i); + state_vector[i] = i; } // Compute the state transition vector @@ -577,7 +528,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ // Initialize the state transition vector passed on to the second stage #pragma unroll for (int32_t i = 0; i < NUM_STATES; ++i) { - out_state_vector.Set(i, state_vector.Get(i)); + out_state_vector.Set(i, state_vector[i]); } // Write out state-transition vector @@ -585,10 +536,10 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ d_thread_state_transition[blockIdx.x * BLOCK_THREADS + threadIdx.x] = out_state_vector; } } + // Stage 2: Perform FSM simulation if ((!IS_TRANS_VECTOR_PASS) || IS_SINGLE_PASS) { - constexpr uint32_t SINGLE_ITEM_COUNT = 1; - MultiItemStateVector<int32_t, SINGLE_ITEM_COUNT> state; + int32_t state = 0; //------------------------------------------------------------------------------ // SINGLE-PASS: @@ -637,10 +588,9 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ .ExclusiveScan(out_state_vector, out_state_vector, state_vector_scan_op, prefix_op); } __syncthreads(); - state.Set(0, out_state_vector.Get(seed_state)); + state = out_state_vector.Get(seed_state); } else { - state.Set( - 0, d_thread_state_transition[blockIdx.x * BLOCK_THREADS + threadIdx.x].Get(seed_state)); + state = d_thread_state_transition[blockIdx.x * BLOCK_THREADS + threadIdx.x].Get(seed_state); } // Perform finite-state machine simulation, computing size of transduced output @@ -649,8 +599,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ TransducedIndexOutItT> callback_wrapper(transducer_table, transduced_out_it, transduced_out_idx_it); - MultiItemStateVector<int32_t, SINGLE_ITEM_COUNT> t_start_state; - t_start_state.Set(0, state.Get(seed_state)); + int32_t t_start_state = state; agent_dfa.GetThreadStateTransitions(symbol_matcher, transition_table, d_chars, @@ -661,6 +610,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ cub::Int2Type<IS_SINGLE_PASS>()); __syncthreads(); + using OffsetPrefixScanCallbackOpT_ = cub::TilePrefixCallbackOp<OffsetT, cub::Sum, OutOffsetScanTileState>; From e6f8defa0b79d040eb465cb76a12af194d1ff899 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 11 Jul 2022 09:06:01 -0700 Subject: [PATCH 17/81] some west-const remainders & unifies StateIndexT --- cpp/src/io/fst/agent_dfa.cuh | 107 +++++++++++++++++------------------ 1 file changed, 53 insertions(+), 54 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index 65918a33e5e..36de79a0757 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -21,6 +21,9 @@ namespace cudf::io::fst::detail { +/// Type used to enumerate (and index) into the states defined by a DFA +using StateIndexT = uint32_t; + //----------------------------------------------------------------------------- // DFA-SIMULATION STATE COMPOSITION FUNCTORS //----------------------------------------------------------------------------- @@ -105,7 +108,7 @@ template <int32_t NUM_INSTANCES, typename TransitionTableT> class StateVectorTransitionOp { public: __host__ __device__ __forceinline__ StateVectorTransitionOp( - TransitionTableT const& transition_table, std::array<int32_t, NUM_INSTANCES>& state_vector) + TransitionTableT const& transition_table, std::array<StateIndexT, NUM_INSTANCES>& state_vector) : transition_table(transition_table), state_vector(state_vector) { } @@ -120,18 +123,18 @@ class StateVectorTransitionOp { } public: - std::array<int32_t, NUM_INSTANCES>& state_vector; - const TransitionTableT& transition_table; + std::array<StateIndexT, NUM_INSTANCES>& state_vector; + TransitionTableT const& transition_table; }; template <typename CallbackOpT, typename TransitionTableT> struct StateTransitionOp { - int32_t state; - const TransitionTableT& transition_table; + StateIndexT state; + TransitionTableT const& transition_table; CallbackOpT& callback_op; __host__ __device__ __forceinline__ StateTransitionOp(TransitionTableT const& transition_table, - int32_t state, + StateIndexT state, CallbackOpT& callback_op) : transition_table(transition_table), state(state), callback_op(callback_op) { @@ -142,7 +145,7 @@ struct StateTransitionOp { SymbolIndexT const& read_symbol_id) { // Remember what state we were in before we made the transition - int32_t previous_state = state; + StateIndexT previous_state = state; state = transition_table(state, read_symbol_id); callback_op.ReadSymbol(character_index, previous_state, state, read_symbol_id); @@ -152,7 +155,6 @@ struct StateTransitionOp { template <typename AgentDFAPolicy, typename SymbolItT, typename OffsetT> struct AgentDFA { using SymbolIndexT = uint32_t; - using StateIndexT = uint32_t; using AliasedLoadT = uint32_t; using CharT = typename std::iterator_traits<SymbolItT>::value_type; @@ -200,14 +202,13 @@ struct AgentDFA { { } - template <int32_t NUM_SYMBOLS, // The net (excluding overlap) number of characters to be parsed - typename SymbolMatcherT, // The symbol matcher returning the matched symbol and its - // length - typename CallbackOpT, // Callback operator + template <int32_t NUM_SYMBOLS, + typename SymbolMatcherT, + typename CallbackOpT, int32_t IS_FULL_BLOCK> - __device__ __forceinline__ static void ThreadParse(const SymbolMatcherT& symbol_matcher, - const CharT* chars, - const SymbolIndexT& max_num_chars, + __device__ __forceinline__ static void ThreadParse(SymbolMatcherT const& symbol_matcher, + CharT const* chars, + SymbolIndexT const& max_num_chars, CallbackOpT callback_op, cub::Int2Type<IS_FULL_BLOCK> /*IS_FULL_BLOCK*/) { @@ -215,7 +216,7 @@ struct AgentDFA { #pragma unroll for (int32_t i = 0; i < NUM_SYMBOLS; ++i) { if (IS_FULL_BLOCK || threadIdx.x * SYMBOLS_PER_THREAD + i < max_num_chars) { - uint32_t matched_id = symbol_matcher(chars[i]); + auto matched_id = symbol_matcher(chars[i]); callback_op.ReadSymbol(i, matched_id); } } @@ -226,9 +227,9 @@ struct AgentDFA { typename StateTransitionOpT, int32_t IS_FULL_BLOCK> __device__ __forceinline__ void GetThreadStateTransitions( - const SymbolMatcherT& symbol_matcher, - const CharT* chars, - const SymbolIndexT& max_num_chars, + SymbolMatcherT const& symbol_matcher, + CharT const* chars, + SymbolIndexT const& max_num_chars, StateTransitionOpT& state_transition_op, cub::Int2Type<IS_FULL_BLOCK> /*IS_FULL_BLOCK*/) { @@ -239,15 +240,15 @@ struct AgentDFA { //--------------------------------------------------------------------- // LOADING FULL BLOCK OF CHARACTERS, NON-ALIASED //--------------------------------------------------------------------- - __device__ __forceinline__ void LoadBlock(const CharT* d_chars, - const OffsetT block_offset, - const OffsetT num_total_symbols, + __device__ __forceinline__ void LoadBlock(CharT const* d_chars, + OffsetT const block_offset, + OffsetT const num_total_symbols, cub::Int2Type<true> /*IS_FULL_BLOCK*/, cub::Int2Type<1> /*ALIGNMENT*/) { CharT thread_chars[SYMBOLS_PER_THREAD]; - const CharT* d_block_symbols = d_chars + block_offset; + CharT const* d_block_symbols = d_chars + block_offset; cub::LoadDirectStriped<BLOCK_THREADS>(threadIdx.x, d_block_symbols, thread_chars); #pragma unroll @@ -259,9 +260,9 @@ struct AgentDFA { //--------------------------------------------------------------------- // LOADING PARTIAL BLOCK OF CHARACTERS, NON-ALIASED //--------------------------------------------------------------------- - __device__ __forceinline__ void LoadBlock(const CharT* d_chars, - const OffsetT block_offset, - const OffsetT num_total_symbols, + __device__ __forceinline__ void LoadBlock(CharT const* d_chars, + OffsetT const block_offset, + OffsetT const num_total_symbols, cub::Int2Type<false> /*IS_FULL_BLOCK*/, cub::Int2Type<1> /*ALIGNMENT*/) { @@ -272,7 +273,7 @@ struct AgentDFA { // Last unit to be loaded is IDIV_CEIL(#SYM, SYMBOLS_PER_UNIT) OffsetT num_total_chars = num_total_symbols - block_offset; - const CharT* d_block_symbols = d_chars + block_offset; + CharT const* d_block_symbols = d_chars + block_offset; cub::LoadDirectStriped<BLOCK_THREADS>( threadIdx.x, d_block_symbols, thread_chars, num_total_chars); @@ -285,16 +286,16 @@ struct AgentDFA { //--------------------------------------------------------------------- // LOADING FULL BLOCK OF CHARACTERS, ALIASED //--------------------------------------------------------------------- - __device__ __forceinline__ void LoadBlock(const CharT* d_chars, - const OffsetT block_offset, - const OffsetT num_total_symbols, + __device__ __forceinline__ void LoadBlock(CharT const* d_chars, + OffsetT const block_offset, + OffsetT const num_total_symbols, cub::Int2Type<true> /*IS_FULL_BLOCK*/, cub::Int2Type<sizeof(AliasedLoadT)> /*ALIGNMENT*/) { AliasedLoadT thread_units[UINTS_PER_THREAD]; - const AliasedLoadT* d_block_symbols = - reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); + AliasedLoadT const* d_block_symbols = + reinterpret_cast<AliasedLoadT const*>(d_chars + block_offset); cub::LoadDirectStriped<BLOCK_THREADS>(threadIdx.x, d_block_symbols, thread_units); #pragma unroll @@ -306,9 +307,9 @@ struct AgentDFA { //--------------------------------------------------------------------- // LOADING PARTIAL BLOCK OF CHARACTERS, ALIASED //--------------------------------------------------------------------- - __device__ __forceinline__ void LoadBlock(const CharT* d_chars, - const OffsetT block_offset, - const OffsetT num_total_symbols, + __device__ __forceinline__ void LoadBlock(CharT const* d_chars, + OffsetT const block_offset, + OffsetT const num_total_symbols, cub::Int2Type<false> /*IS_FULL_BLOCK*/, cub::Int2Type<sizeof(AliasedLoadT)> /*ALIGNMENT*/) { @@ -320,8 +321,8 @@ struct AgentDFA { OffsetT num_total_units = CUB_QUOTIENT_CEILING(num_total_symbols - block_offset, sizeof(AliasedLoadT)); - const AliasedLoadT* d_block_symbols = - reinterpret_cast<const AliasedLoadT*>(d_chars + block_offset); + AliasedLoadT const* d_block_symbols = + reinterpret_cast<AliasedLoadT const*>(d_chars + block_offset); cub::LoadDirectStriped<BLOCK_THREADS>( threadIdx.x, d_block_symbols, thread_units, num_total_units); @@ -334,9 +335,9 @@ struct AgentDFA { //--------------------------------------------------------------------- // LOADING BLOCK OF CHARACTERS: DISPATCHER //--------------------------------------------------------------------- - __device__ __forceinline__ void LoadBlock(const CharT* d_chars, - const OffsetT block_offset, - const OffsetT num_total_symbols) + __device__ __forceinline__ void LoadBlock(CharT const* d_chars, + OffsetT const block_offset, + OffsetT const num_total_symbols) { // Check if pointer is aligned to four bytes if (((uintptr_t)(const void*)(d_chars + block_offset) % 4) == 0) { @@ -360,12 +361,12 @@ struct AgentDFA { template <int32_t NUM_STATES, typename SymbolMatcherT, typename TransitionTableT> __device__ __forceinline__ void GetThreadStateTransitionVector( - const SymbolMatcherT& symbol_matcher, - const TransitionTableT& transition_table, - const CharT* d_chars, - const OffsetT block_offset, - const OffsetT num_total_symbols, - std::array<int32_t, NUM_STATES>& state_vector) + SymbolMatcherT const& symbol_matcher, + TransitionTableT const& transition_table, + CharT const* d_chars, + OffsetT const block_offset, + OffsetT const num_total_symbols, + std::array<StateIndexT, NUM_STATES>& state_vector) { using StateVectorTransitionOpT = StateVectorTransitionOp<NUM_STATES, TransitionTableT>; @@ -405,7 +406,7 @@ struct AgentDFA { CharT const* d_chars, OffsetT const block_offset, OffsetT const num_total_symbols, - int32_t& state, + StateIndexT& state, CallbackOpT& callback_op, cub::Int2Type<BYPASS_LOAD> /**/) { @@ -459,7 +460,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ void SimulateDFAKernel(DfaT dfa, SymbolItT d_chars, OffsetT const num_chars, - uint32_t seed_state, + StateIndexT seed_state, StateVectorT* __restrict__ d_thread_state_transition, TileStateT tile_state, OutOffsetScanTileState offset_tile_state, @@ -467,11 +468,9 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ TransducedIndexOutItT transduced_out_idx_it, TransducedCountOutItT d_num_transduced_out_it) { - using StateIndexT = uint32_t; - using AgentDfaSimT = AgentDFA<AgentDFAPolicy, SymbolItT, OffsetT>; - static constexpr uint32_t NUM_STATES = DfaT::MAX_NUM_STATES; + static constexpr int32_t NUM_STATES = DfaT::MAX_NUM_STATES; enum { BLOCK_THREADS = AgentDFAPolicy::BLOCK_THREADS, @@ -509,7 +508,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ // Stage 1: Compute the state-transition vector if (IS_TRANS_VECTOR_PASS || IS_SINGLE_PASS) { // Keeping track of the state for each of the <NUM_STATES> state machines - std::array<int32_t, NUM_STATES> state_vector; + std::array<StateIndexT, NUM_STATES> state_vector; // Initialize the seed state transition vector with the identity vector #pragma unroll @@ -539,7 +538,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ // Stage 2: Perform FSM simulation if ((!IS_TRANS_VECTOR_PASS) || IS_SINGLE_PASS) { - int32_t state = 0; + StateIndexT state = 0; //------------------------------------------------------------------------------ // SINGLE-PASS: @@ -599,7 +598,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ TransducedIndexOutItT> callback_wrapper(transducer_table, transduced_out_it, transduced_out_idx_it); - int32_t t_start_state = state; + StateIndexT t_start_state = state; agent_dfa.GetThreadStateTransitions(symbol_matcher, transition_table, d_chars, From a798852ea24e44a8432847fae82f4009cce20c05 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 11 Jul 2022 11:00:22 -0700 Subject: [PATCH 18/81] adds check for state transition narrowing conversion --- cpp/src/io/fst/agent_dfa.cuh | 29 ++++++++++++++++++++--------- cpp/src/io/fst/lookup_tables.cuh | 6 +++++- cpp/tests/io/fst/fst_test.cu | 31 +++++++++++++++---------------- 3 files changed, 40 insertions(+), 26 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index 36de79a0757..7eff0c50024 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -24,9 +24,6 @@ namespace cudf::io::fst::detail { /// Type used to enumerate (and index) into the states defined by a DFA using StateIndexT = uint32_t; -//----------------------------------------------------------------------------- -// DFA-SIMULATION STATE COMPOSITION FUNCTORS -//----------------------------------------------------------------------------- /** * @brief Implements an associative composition operation for state transition vectors and * offset-to-overap vectors to be used with a prefix scan. @@ -55,9 +52,18 @@ struct VectorCompositeOp { } }; -//----------------------------------------------------------------------------- -// DFA-SIMULATION CALLBACK WRAPPERS/HELPERS -//----------------------------------------------------------------------------- +/** + * @brief A class whose ReadSymbol member function is invoked for each symbol being read from the + * input tape. The wrapper class looks up whether a state transition caused by a symbol is supposed + * to emit any output symbol (the "transduced" output) and, if so, keeps track of how many symbols + * it intends to write out and writing out such symbols to the given output iterators. + * + * @tparam TransducerTableT The type implementing a transducer table that can be used for looking up + * the symbols that are supposed to be emitted on a given state transition. + * @tparam TransducedOutItT A Random-access output iterator type to which symbols returned by the + * transducer table are assignable. + * @tparam TransducedIndexOutItT A Random-access output iterator type to which indexes are written. + */ template <typename TransducerTableT, typename TransducedOutItT, typename TransducedIndexOutItT> class DFASimulationCallbackWrapper { public: @@ -101,9 +107,14 @@ class DFASimulationCallbackWrapper { bool write; }; -//----------------------------------------------------------------------------- -// STATE-TRANSITION CALLBACKS -//----------------------------------------------------------------------------- +/** + * @brief Helper class that transitions the state of multiple DFA instances simultaneously whenever + * a symbol is read. + * + * @tparam NUM_INSTANCES The number of DFA instances to keep track of + * @tparam TransitionTableT The transition table type used for looking up the new state for a + * current_state and a read_symbol. + */ template <int32_t NUM_INSTANCES, typename TransitionTableT> class StateVectorTransitionOp { public: diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index f1dd31d1f4b..a0a9f81a302 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -168,13 +168,17 @@ class TransitionTable { ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; }; + template <typename StateIdT, typename = std::void_t<decltype(ItemT{std::declval<StateIdT>()})>> static void InitDeviceTransitionTable(hostdevice_vector<KernelParameter>& transition_table_init, - const std::vector<std::vector<int>>& trans_table, + std::vector<std::vector<StateIdT>> const& trans_table, rmm::cuda_stream_view stream) { // trans_table[state][symbol] -> new state for (std::size_t state = 0; state < trans_table.size(); ++state) { for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { + CUDF_EXPECTS( + trans_table[state][symbol] <= std::numeric_limits<ItemT>::max(), + "Target state index value exceeds value representable by the transition table's type"); transition_table_init.host_ptr()->transitions[symbol * MAX_NUM_STATES + state] = trans_table[state][symbol]; } diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 0c337be61f3..e1ee655f30b 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -118,21 +118,20 @@ static std::pair<OutputItT, IndexOutputItT> fst_baseline(InputItT begin, //------------------------------------------------------------------------------ // TEST FST SPECIFICATIONS //------------------------------------------------------------------------------ -// FST to check for brackets and braces outside of pairs of quotes -// The state being active while being outside of a string. When encountering an opening bracket -// or curly brace, we push it onto the stack. When encountering a closing bracket or brace, we -// pop it from the stack. -constexpr uint32_t TT_OOS = 0U; - -// The state being active while being within a string (e.g., field name or a string value). We do -// not push or pop from the stack while being in this state. -constexpr uint32_t TT_STR = 1U; - -// The state being active after encountering an escape symbol (e.g., '\') while being in the TT_STR -// state. constexpr uint32_t TT_ESC = 2U; // cmt to avoid 'unused' warning - -// Total number of states -constexpr uint32_t TT_NUM_STATES = 3U; +enum DFA_STATES : char { + // The state being active while being outside of a string. When encountering an opening bracket or + // curly brace, we push it onto the stack. When encountering a closing bracket or brace, we pop it + // from the stack. + TT_OOS = 0U, + // The state being active while being within a string (e.g., field name or a string value). We do + // not push or pop from the stack while being in this state. + TT_STR, + // The state being active after encountering an escape symbol (e.g., '\') while being in the + // TT_STR state. + TT_ESC [[maybe_unused]], + // Total number of states + TT_NUM_STATES +}; // Definition of the symbol groups enum PDA_SG_ID { @@ -147,7 +146,7 @@ enum PDA_SG_ID { }; // Transition table -const std::vector<std::vector<int32_t>> pda_state_tt = { +const std::vector<std::vector<char>> pda_state_tt = { /* IN_STATE { [ } ] " \ OTHER */ /* TT_OOS */ {TT_OOS, TT_OOS, TT_OOS, TT_OOS, TT_STR, TT_OOS, TT_OOS}, /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_STR, TT_STR}, From eb2496205ce65c808e968348c22e35862bb19ff7 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Tue, 12 Jul 2022 04:52:36 -0700 Subject: [PATCH 19/81] fixes logical stack test includes --- cpp/tests/io/fst/logical_stack_test.cu | 1 - 1 file changed, 1 deletion(-) diff --git a/cpp/tests/io/fst/logical_stack_test.cu b/cpp/tests/io/fst/logical_stack_test.cu index 3c2cdd7fb5c..d76ff16f85d 100644 --- a/cpp/tests/io/fst/logical_stack_test.cu +++ b/cpp/tests/io/fst/logical_stack_test.cu @@ -22,7 +22,6 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> -#include <rmm/device_buffer.hpp> #include <rmm/device_uvector.hpp> #include <cstdlib> From f52e61457b2b88b8b6a4f61bfd214283ea2d28a9 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 14 Jul 2022 09:16:18 -0700 Subject: [PATCH 20/81] replaces enum with typed constexpr --- cpp/src/io/fst/agent_dfa.cuh | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index 7eff0c50024..788e455592b 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -483,11 +483,8 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ static constexpr int32_t NUM_STATES = DfaT::MAX_NUM_STATES; - enum { - BLOCK_THREADS = AgentDFAPolicy::BLOCK_THREADS, - ITEMS_PER_THREAD = AgentDFAPolicy::ITEMS_PER_THREAD, - SYMBOLS_PER_BLOCK = AgentDfaSimT::SYMBOLS_PER_BLOCK - }; + constexpr uint32_t BLOCK_THREADS = AgentDFAPolicy::BLOCK_THREADS; + constexpr uint32_t SYMBOLS_PER_BLOCK = AgentDfaSimT::SYMBOLS_PER_BLOCK; // Shared memory required by the DFA simulation algorithm __shared__ typename AgentDfaSimT::TempStorage dfa_storage; From 3038058e48347ef95a18fe2f18190c5e0de7c9a0 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 14 Jul 2022 09:17:09 -0700 Subject: [PATCH 21/81] adds excplitis error checking --- cpp/src/io/fst/dispatch_dfa.cuh | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/cpp/src/io/fst/dispatch_dfa.cuh b/cpp/src/io/fst/dispatch_dfa.cuh index 6de1757bfed..57e62608841 100644 --- a/cpp/src/io/fst/dispatch_dfa.cuh +++ b/cpp/src/io/fst/dispatch_dfa.cuh @@ -168,7 +168,7 @@ struct DispatchFSM : DeviceFSMPolicy { // Get PTX version int ptx_version; error = cub::PtxVersion(ptx_version); - if (error) return error; + if (error != cudaSuccess) return error; // Create dispatch functor DispatchFSM dispatch(d_temp_storage, @@ -310,15 +310,15 @@ struct DispatchFSM : DeviceFSMPolicy { cudaError_t error = cudaSuccess; // Get SM count - int device_ordinal; - int sm_count; + int device_ordinal = -1; + int sm_count = -1; // Get current device error = cudaGetDevice(&device_ordinal); - if (error) + if (error != cudaSuccess)return error; - error = cudaDeviceGetAttribute(&sm_count, cudaDevAttrMultiProcessorCount, device_ordinal); - if (error) return error; + error = cudaDeviceGetAttribute(&sm_count, cudaDevAttrMultiProcessorCount, device_ordinal); + if (error != cudaSuccess) return error; //------------------------------------------------------------------------------ // DERIVED TYPEDEFS @@ -377,21 +377,21 @@ struct DispatchFSM : DeviceFSMPolicy { // Bytes needed for tile status descriptors (fusing state-transition vector + DFA simulation) if (SINGLE_PASS_STV) { error = ScanTileStateT::AllocationSize(num_blocks, allocation_sizes[MEM_SINGLE_PASS_STV]); - if (error) return error; + if (error != cudaSuccess) return error; } // Bytes needed for tile status descriptors (DFA simulation pass for output size computation + // output-generating pass) if (IS_FST) { error = FstScanTileStateT::AllocationSize(num_blocks, allocation_sizes[MEM_FST_OFFSET]); - if (error) return error; + if (error != cudaSuccess) return error; } // Alias the temporary allocations from the single storage blob (or compute the necessary size // of the blob) error = cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); - if (error) return error; + if (error != cudaSuccess) return error; // Return if the caller is simply requesting the size of the storage allocation if (d_temp_storage == NULL) return cudaSuccess; @@ -408,7 +408,7 @@ struct DispatchFSM : DeviceFSMPolicy { // Construct the tile status (aliases memory internally et al.) error = fst_offset_tile_state.Init( num_blocks, allocations[MEM_FST_OFFSET], allocation_sizes[MEM_FST_OFFSET]); - if (error) return error; + if (error != cudaSuccess) return error; constexpr uint32_t FST_INIT_TPB = 256; uint32_t num_fst_init_blocks = CUB_QUOTIENT_CEILING(num_blocks, FST_INIT_TPB); initialization_pass_kernel<<<num_fst_init_blocks, FST_INIT_TPB, 0, stream>>>( @@ -423,7 +423,7 @@ struct DispatchFSM : DeviceFSMPolicy { // Construct the tile status (aliases memory internally et al.) error = stv_tile_state.Init( num_blocks, allocations[MEM_SINGLE_PASS_STV], allocation_sizes[MEM_SINGLE_PASS_STV]); - if (error) return error; + if (error != cudaSuccess) return error; constexpr uint32_t STV_INIT_TPB = 256; uint32_t num_stv_init_blocks = CUB_QUOTIENT_CEILING(num_blocks, STV_INIT_TPB); initialization_pass_kernel<<<num_stv_init_blocks, STV_INIT_TPB, 0, stream>>>(stv_tile_state, From d351e5c4197acf7c7ab215ea7555926cb2d1f5b8 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 14 Jul 2022 09:17:59 -0700 Subject: [PATCH 22/81] addresses style review comments & fixes a todo --- cpp/src/io/fst/lookup_tables.cuh | 30 ++++++++++-------------------- 1 file changed, 10 insertions(+), 20 deletions(-) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index a0a9f81a302..208890d28d3 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -41,7 +41,7 @@ class SingleSymbolSmemLUT { // Type used for representing a symbol group id (i.e., what we return for a given symbol) using SymbolGroupIdT = uint8_t; - /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) + // Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); struct _TempStorage { @@ -60,9 +60,6 @@ class SingleSymbolSmemLUT { using TempStorage = cub::Uninitialized<_TempStorage>; - //------------------------------------------------------------------------------ - // HELPER METHODS - //------------------------------------------------------------------------------ /** * @brief * @@ -104,21 +101,14 @@ class SingleSymbolSmemLUT { sgid_init.host_ptr()->sym_to_sgid[max_base_match_val + 1] = no_match_id; // Alias memory / return memory requiremenets - // TODO I think this could be +1? - sgid_init.host_ptr()->num_valid_entries = max_base_match_val + 2; + sgid_init.host_ptr()->num_valid_entries = max_base_match_val + 1; sgid_init.host_to_device(stream); } - //------------------------------------------------------------------------------ - // MEMBER VARIABLES - //------------------------------------------------------------------------------ _TempStorage& temp_storage; SymbolGroupIdT num_valid_entries; - //------------------------------------------------------------------------------ - // CONSTRUCTOR - //------------------------------------------------------------------------------ __device__ __forceinline__ _TempStorage& PrivateStorage() { __shared__ _TempStorage private_storage; @@ -170,17 +160,17 @@ class TransitionTable { template <typename StateIdT, typename = std::void_t<decltype(ItemT{std::declval<StateIdT>()})>> static void InitDeviceTransitionTable(hostdevice_vector<KernelParameter>& transition_table_init, - std::vector<std::vector<StateIdT>> const& trans_table, + std::vector<std::vector<StateIdT>> const& translation_table, rmm::cuda_stream_view stream) { - // trans_table[state][symbol] -> new state - for (std::size_t state = 0; state < trans_table.size(); ++state) { - for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { + // translation_table[state][symbol] -> new state + for (std::size_t state = 0; state < translation_table.size(); ++state) { + for (std::size_t symbol = 0; symbol < translation_table[state].size(); ++symbol) { CUDF_EXPECTS( - trans_table[state][symbol] <= std::numeric_limits<ItemT>::max(), + translation_table[state][symbol] <= std::numeric_limits<ItemT>::max(), "Target state index value exceeds value representable by the transition table's type"); transition_table_init.host_ptr()->transitions[symbol * MAX_NUM_STATES + state] = - trans_table[state][symbol]; + translation_table[state][symbol]; } } @@ -314,7 +304,7 @@ class TransducerLookupTable { */ static void InitDeviceTranslationTable( hostdevice_vector<KernelParameter>& translation_table_init, - std::vector<std::vector<std::vector<OutSymbolT>>> const& trans_table, + std::vector<std::vector<std::vector<OutSymbolT>>> const& translation_table, rmm::cuda_stream_view stream) { std::vector<OutSymbolT> out_symbols; @@ -324,7 +314,7 @@ class TransducerLookupTable { out_symbol_offsets.push_back(0); // Iterate over the states in the transition table - for (auto const& state_trans : trans_table) { + for (auto const& state_trans : translation_table) { uint32_t num_added = 0; // Iterate over the symbols in the transition table for (auto const& symbol_out : state_trans) { From 3f479528b2b24b6eddae4d622509f8392b97eb0d Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 14 Jul 2022 09:22:03 -0700 Subject: [PATCH 23/81] replaces gtest asserts with expects --- cpp/tests/io/fst/fst_test.cu | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index e1ee655f30b..5d169cd9ac1 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -255,10 +255,10 @@ TEST_F(FstTest, GroundTruth) // Verify results ASSERT_EQ(output_gpu_size[0], output_cpu.size()); for (std::size_t i = 0; i < output_cpu.size(); i++) { - ASSERT_EQ(output_gpu[i], output_cpu[i]) << "Mismatch at index #" << i; + EXPECT_EQ(output_gpu[i], output_cpu[i]) << "Mismatch at index #" << i; } for (std::size_t i = 0; i < output_cpu.size(); i++) { - ASSERT_EQ(out_indexes_gpu[i], out_index_cpu[i]) << "Mismatch at index #" << i; + EXPECT_EQ(out_indexes_gpu[i], out_index_cpu[i]) << "Mismatch at index #" << i; } } From cba16196b356ecc807a6ae67a20b357677cf26a4 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 14 Jul 2022 09:31:12 -0700 Subject: [PATCH 24/81] fixes style in dispatch dfa --- cpp/src/io/fst/dispatch_dfa.cuh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/fst/dispatch_dfa.cuh b/cpp/src/io/fst/dispatch_dfa.cuh index 57e62608841..2a5594c383d 100644 --- a/cpp/src/io/fst/dispatch_dfa.cuh +++ b/cpp/src/io/fst/dispatch_dfa.cuh @@ -311,11 +311,11 @@ struct DispatchFSM : DeviceFSMPolicy { // Get SM count int device_ordinal = -1; - int sm_count = -1; + int sm_count = -1; // Get current device error = cudaGetDevice(&device_ordinal); - if (error != cudaSuccess)return error; + if (error != cudaSuccess) return error; error = cudaDeviceGetAttribute(&sm_count, cudaDevAttrMultiProcessorCount, device_ordinal); if (error != cudaSuccess) return error; From bea2a02226314cddb6073726d2feafa21d89bb52 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Fri, 15 Jul 2022 01:54:20 -0700 Subject: [PATCH 25/81] replaces vanilla loop with iota --- cpp/src/io/fst/agent_dfa.cuh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index 788e455592b..77f04555dc8 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -19,6 +19,9 @@ #include <cub/cub.cuh> +#include <thrust/execution_policy.h> +#include <thrust/sequence.h> + namespace cudf::io::fst::detail { /// Type used to enumerate (and index) into the states defined by a DFA @@ -519,10 +522,7 @@ __launch_bounds__(int32_t(AgentDFAPolicy::BLOCK_THREADS)) __global__ std::array<StateIndexT, NUM_STATES> state_vector; // Initialize the seed state transition vector with the identity vector -#pragma unroll - for (int32_t i = 0; i < NUM_STATES; ++i) { - state_vector[i] = i; - } + thrust::sequence(thrust::seq, std::begin(state_vector), std::end(state_vector)); // Compute the state transition vector agent_dfa.GetThreadStateTransitionVector<NUM_STATES>(symbol_matcher, From 8a184e97ce14a400f70d463f7e55f95f32d7a547 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Fri, 15 Jul 2022 22:51:18 -0700 Subject: [PATCH 26/81] rephrases documentation on in-reg array --- cpp/src/io/fst/in_reg_array.cuh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index 9d59d04cb9b..352d7871699 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -51,7 +51,7 @@ class MultiFragmentInRegArray { static constexpr uint32_t BITS_PER_FRAG_ITEM = 0x01U << (cub::Log2<(AVAIL_BITS_PER_FRAG_ITEM + 1)>::VALUE - 1); - // Number of fragments required to store and to reconstruct each item + // The total number of fragments required to store all the items static constexpr uint32_t FRAGMENTS_PER_ITEM = (MIN_BITS_PER_ITEM + BITS_PER_FRAG_ITEM - 1) / BITS_PER_FRAG_ITEM; From 4783aae2aabbd03f9a439822ddd02d0328b5d52a Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 06:28:05 -0700 Subject: [PATCH 27/81] improves style in fst test --- cpp/tests/io/fst/fst_test.cu | 26 +++++++++++--------------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 5d169cd9ac1..606fe6e7bcd 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -89,22 +89,18 @@ static std::pair<OutputItT, IndexOutputItT> fst_baseline(InputItT begin, // The symbol currently being read auto const& symbol = *it; - std::size_t symbol_group = 0; - // Iterate over symbol groups and search for the first symbol group containing the current - // symbol - for (auto const& sg : symbol_group_lut) { - if (std::find(std::cbegin(sg), std::cend(sg), symbol) != std::cend(sg)) { break; } - symbol_group++; - } + // symbol, if no match is found we use cend(symbol_group_lut) as the "catch-all" symbol group + auto symbol_group_it = std::find_if(std::cbegin(symbol_group_lut), std::cend(symbol_group_lut), + [symbol](auto& sg) { + return std::find(std::cbegin(sg), std::cend(sg), symbol) != std::cend(sg); + }); + auto symbol_group = std::distance(std::cbegin(symbol_group_lut), symbol_group_it); // Output the translated symbols to the output tape - for (auto out : translation_table[state][symbol_group]) { - *out_tape = out; - ++out_tape; - *out_index_tape = in_offset; - out_index_tape++; - } + out_tape = std::copy(std::cbegin(translation_table[state][symbol_group]), std::cend(translation_table[state][symbol_group]), out_tape); + auto out_size = std::distance(std::cbegin(translation_table[state][symbol_group]), std::cend(translation_table[state][symbol_group])); + out_index_tape = std::fill_n(out_index_tape, out_size, in_offset); // Transition the state of the finite-state machine state = transition_table[state][symbol_group]; @@ -128,7 +124,7 @@ enum DFA_STATES : char { TT_STR, // The state being active after encountering an escape symbol (e.g., '\') while being in the // TT_STR state. - TT_ESC [[maybe_unused]], + TT_ESC, // Total number of states TT_NUM_STATES }; @@ -149,7 +145,7 @@ enum PDA_SG_ID { const std::vector<std::vector<char>> pda_state_tt = { /* IN_STATE { [ } ] " \ OTHER */ /* TT_OOS */ {TT_OOS, TT_OOS, TT_OOS, TT_OOS, TT_STR, TT_OOS, TT_OOS}, - /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_STR, TT_STR}, + /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_ESC, TT_STR}, /* TT_ESC */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR}}; // Translation table (i.e., for each transition, what are the symbols that we output) From 6203709bcfad35d745eb5cae77ddcf6166fa8216 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 10:30:56 -0700 Subject: [PATCH 28/81] adds comments in in_reg array --- cpp/src/io/fst/in_reg_array.cuh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index 352d7871699..f0866ebe3bb 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -15,6 +15,7 @@ */ #pragma once +#include <cudf/detail/utilities/integer_utils.hpp> #include <cudf/types.hpp> #include <cub/cub.cuh> @@ -52,12 +53,14 @@ class MultiFragmentInRegArray { 0x01U << (cub::Log2<(AVAIL_BITS_PER_FRAG_ITEM + 1)>::VALUE - 1); // The total number of fragments required to store all the items - static constexpr uint32_t FRAGMENTS_PER_ITEM = - (MIN_BITS_PER_ITEM + BITS_PER_FRAG_ITEM - 1) / BITS_PER_FRAG_ITEM; + static constexpr uint32_t FRAGMENTS_PER_ITEM = cudf::util::div_rounding_up_safe(MIN_BITS_PER_ITEM, BITS_PER_FRAG_ITEM); //------------------------------------------------------------------------------ // HELPER FUNCTIONS //------------------------------------------------------------------------------ + /** + * @brief Returns the \p num_bits bits starting at \p bit_start + */ CUDF_HOST_DEVICE uint32_t bfe(const uint32_t& data, uint32_t bit_start, uint32_t num_bits) const { #if CUB_PTX_ARCH > 0 @@ -68,6 +71,9 @@ class MultiFragmentInRegArray { #endif } +/** + * @brief Replaces the \p num_bits bits in \p data starting from \p bit_start with the lower \p num_bits from \p bits. + */ CUDF_HOST_DEVICE void bfi(uint32_t& data, uint32_t bits, uint32_t bit_start, From ad5817a4a9b3f12127926f95edd958d9fdb1a4d7 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 10:34:59 -0700 Subject: [PATCH 29/81] adds comments to lookup tables --- cpp/src/io/fst/lookup_tables.cuh | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index 208890d28d3..279baef939d 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -31,7 +31,7 @@ namespace cudf::io::fst::detail { /** * @brief Class template that can be plugged into the finite-state machine to look up the symbol * group index for a given symbol. Class template does not support multi-symbol lookups (i.e., no - * look-ahead). + * look-ahead). The class uses shared memory for the lookups. * * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id */ @@ -61,7 +61,7 @@ class SingleSymbolSmemLUT { using TempStorage = cub::Uninitialized<_TempStorage>; /** - * @brief + * @brief Initializes the given \p sgid_init with the symbol group lookups defined by \p symbol_strings. * * @param[out] sgid_init A hostdevice_vector that will be populated * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols @@ -128,9 +128,7 @@ class SingleSymbolSmemLUT { #else // CPU-side init - for (std::size_t i = 0; i < kernel_param.num_luts; i++) { - this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; - } + std::copy_n(kernel_param.sym_to_sgid, kernel_param.num_luts, this->temp_storage.sym_to_sgid); #endif } @@ -271,7 +269,7 @@ class dfa_device_view { /** * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols - * that the finite-state transducer is supposed to output for each transition + * that the finite-state transducer is supposed to output for each transition. The class uses shared memory for the lookups. * * @tparam OutSymbolT The symbol type being output * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols From dc5565359d4aaa92a68d81ff3a74674be2240e15 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 12:03:59 -0700 Subject: [PATCH 30/81] fixes formatting --- cpp/src/io/fst/in_reg_array.cuh | 10 ++++++---- cpp/src/io/fst/lookup_tables.cuh | 6 ++++-- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index f0866ebe3bb..87ea2d3325f 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -53,7 +53,8 @@ class MultiFragmentInRegArray { 0x01U << (cub::Log2<(AVAIL_BITS_PER_FRAG_ITEM + 1)>::VALUE - 1); // The total number of fragments required to store all the items - static constexpr uint32_t FRAGMENTS_PER_ITEM = cudf::util::div_rounding_up_safe(MIN_BITS_PER_ITEM, BITS_PER_FRAG_ITEM); + static constexpr uint32_t FRAGMENTS_PER_ITEM = + cudf::util::div_rounding_up_safe(MIN_BITS_PER_ITEM, BITS_PER_FRAG_ITEM); //------------------------------------------------------------------------------ // HELPER FUNCTIONS @@ -71,9 +72,10 @@ class MultiFragmentInRegArray { #endif } -/** - * @brief Replaces the \p num_bits bits in \p data starting from \p bit_start with the lower \p num_bits from \p bits. - */ + /** + * @brief Replaces the \p num_bits bits in \p data starting from \p bit_start with the lower \p + * num_bits from \p bits. + */ CUDF_HOST_DEVICE void bfi(uint32_t& data, uint32_t bits, uint32_t bit_start, diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index 279baef939d..98ef49d893d 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -61,7 +61,8 @@ class SingleSymbolSmemLUT { using TempStorage = cub::Uninitialized<_TempStorage>; /** - * @brief Initializes the given \p sgid_init with the symbol group lookups defined by \p symbol_strings. + * @brief Initializes the given \p sgid_init with the symbol group lookups defined by \p + * symbol_strings. * * @param[out] sgid_init A hostdevice_vector that will be populated * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols @@ -269,7 +270,8 @@ class dfa_device_view { /** * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols - * that the finite-state transducer is supposed to output for each transition. The class uses shared memory for the lookups. + * that the finite-state transducer is supposed to output for each transition. The class uses shared + * memory for the lookups. * * @tparam OutSymbolT The symbol type being output * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols From 378be9f43e00e80e77b8f685bf4a65ca9a854dcc Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 13:02:53 -0700 Subject: [PATCH 31/81] exchanges loops in favor of copy and fills --- cpp/src/io/fst/lookup_tables.cuh | 41 ++++++++++++++++++++++---------- 1 file changed, 28 insertions(+), 13 deletions(-) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index 98ef49d893d..5884279e4db 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -116,6 +116,12 @@ class SingleSymbolSmemLUT { return private_storage; } + /** + * @brief Initializes the lookup table, primarily to be invoked from within device code but also + * provides host-side implementation for verification. + * @note Synchronizes the thread block, if called from device, and, hence, requires all threads + * of the thread block to call the constructor + */ constexpr CUDF_HOST_DEVICE SingleSymbolSmemLUT(KernelParameter const& kernel_param, TempStorage& temp_storage) : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) @@ -140,6 +146,13 @@ class SingleSymbolSmemLUT { } }; +/** + * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a new target state. The + * class uses shared memory for the lookups. + * + * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition + * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support + */ template <int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES> class TransitionTable { private: @@ -187,9 +200,8 @@ class TransitionTable { } __syncthreads(); #else - for (int i = 0; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i++) { - this->temp_storage.transitions[i] = kernel_param.transitions[i]; - } + std::copy_n( + kernel_param.transitions, MAX_NUM_STATES * MAX_NUM_SYMBOLS, this->temp_storage.transitions); #endif } @@ -299,8 +311,11 @@ class TransducerLookupTable { OutSymbolT d_out_symbols[MAX_TABLE_SIZE]; }; - /** - * @brief Initializes the translation table (both the host and device parts) + /** + * @brief Initializes the lookup table, primarily to be invoked from within device code but also + * provides host-side implementation for verification. + * @note Synchronizes the thread block, if called from device, and, hence, requires all threads + * of the thread block to call the constructor */ static void InitDeviceTranslationTable( hostdevice_vector<KernelParameter>& translation_table_init, @@ -329,7 +344,7 @@ class TransducerLookupTable { if (MAX_NUM_SYMBOLS > num_added) { int32_t count = MAX_NUM_SYMBOLS - num_added; auto begin_it = std::prev(std::end(out_symbol_offsets)); - std::copy(begin_it, begin_it + count, std::back_inserter(out_symbol_offsets)); + std::fill_n(begin_it, count, out_symbol_offsets[0]); } } @@ -359,7 +374,9 @@ class TransducerLookupTable { public: /** - * @brief Synchronizes the thread block, if called from device, and, hence, requires all threads + * @brief Initializes the lookup table, primarily to be invoked from within device code but also + * provides host-side implementation for verification. + * @note Synchronizes the thread block, if called from device, and, hence, requires all threads * of the thread block to call the constructor */ CUDF_HOST_DEVICE TransducerLookupTable(KernelParameter const& kernel_param, @@ -379,12 +396,10 @@ class TransducerLookupTable { } __syncthreads(); #else - for (int i = 0; i < num_offsets; i++) { - this->temp_storage.out_symbol_offsets[i] = kernel_param.d_out_offsets[i]; - } - for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { - this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; - } + std::copy_n(kernel_param.d_out_offsets, num_offsets, this->temp_storage.out_symbol_offsets); + std::copy_n(kernel_param.d_out_symbols, + this->temp_storage.out_symbol_offsets, + this->temp_storage.out_symbols); #endif } From 4ba547227a5e90e01e995df9c76ae2a284548c1b Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 15:00:35 -0700 Subject: [PATCH 32/81] clarifies documentation in agent dfa --- cpp/src/io/fst/agent_dfa.cuh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/fst/agent_dfa.cuh b/cpp/src/io/fst/agent_dfa.cuh index 77f04555dc8..d847598d6dd 100644 --- a/cpp/src/io/fst/agent_dfa.cuh +++ b/cpp/src/io/fst/agent_dfa.cuh @@ -28,8 +28,8 @@ namespace cudf::io::fst::detail { using StateIndexT = uint32_t; /** - * @brief Implements an associative composition operation for state transition vectors and - * offset-to-overap vectors to be used with a prefix scan. + * @brief Implements an associative composition operation for state transition vectors to be used + * with a prefix scan. * * Read the following table as follows: c = op(l,r), where op is the composition operator. * For row 0: l maps 0 to 2. r maps 2 to 2. Hence, the result for 0 is 2. From 7980978a97bb3650a3c16f62c87584a352c6b991 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 15:26:50 -0700 Subject: [PATCH 33/81] disambiguates transition and translation tables --- cpp/src/io/fst/lookup_tables.cuh | 12 +++++++++++- cpp/tests/io/fst/fst_test.cu | 15 +++++++++------ 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index 5884279e4db..341c3b7a51d 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -311,7 +311,7 @@ class TransducerLookupTable { OutSymbolT d_out_symbols[MAX_TABLE_SIZE]; }; - /** + /** * @brief Initializes the lookup table, primarily to be invoked from within device code but also * provides host-side implementation for verification. * @note Synchronizes the thread block, if called from device, and, hence, requires all threads @@ -464,6 +464,16 @@ class Dfa { } public: + /** + * @brief Constructs a new DFA. + * + * @param symbol_vec Sequence container of symbol groups. Each symbol group is a sequence + * container to symbols within that group. The index of the symbol group containing a symbol being + * read will be used as symbol_gid of the transition and translation tables. + * @param tt_vec The transition table + * @param out_tt_vec The translation table + * @param stream The stream to which memory operations and kernels are getting dispatched to + */ template <typename StateIdT, typename SymbolGroupIdItT> Dfa(SymbolGroupIdItT const& symbol_vec, std::vector<std::vector<StateIdT>> const& tt_vec, diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 606fe6e7bcd..757410b7e65 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -91,15 +91,18 @@ static std::pair<OutputItT, IndexOutputItT> fst_baseline(InputItT begin, // Iterate over symbol groups and search for the first symbol group containing the current // symbol, if no match is found we use cend(symbol_group_lut) as the "catch-all" symbol group - auto symbol_group_it = std::find_if(std::cbegin(symbol_group_lut), std::cend(symbol_group_lut), - [symbol](auto& sg) { - return std::find(std::cbegin(sg), std::cend(sg), symbol) != std::cend(sg); - }); + auto symbol_group_it = + std::find_if(std::cbegin(symbol_group_lut), std::cend(symbol_group_lut), [symbol](auto& sg) { + return std::find(std::cbegin(sg), std::cend(sg), symbol) != std::cend(sg); + }); auto symbol_group = std::distance(std::cbegin(symbol_group_lut), symbol_group_it); // Output the translated symbols to the output tape - out_tape = std::copy(std::cbegin(translation_table[state][symbol_group]), std::cend(translation_table[state][symbol_group]), out_tape); - auto out_size = std::distance(std::cbegin(translation_table[state][symbol_group]), std::cend(translation_table[state][symbol_group])); + out_tape = std::copy(std::cbegin(translation_table[state][symbol_group]), + std::cend(translation_table[state][symbol_group]), + out_tape); + auto out_size = std::distance(std::cbegin(translation_table[state][symbol_group]), + std::cend(translation_table[state][symbol_group])); out_index_tape = std::fill_n(out_index_tape, out_size, in_offset); // Transition the state of the finite-state machine From 2bce0616851b47fb7e7cafb39adf15eb28454018 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 20 Jul 2022 21:58:40 -0700 Subject: [PATCH 34/81] minor style fix --- cpp/tests/io/fst/fst_test.cu | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 757410b7e65..40866d8bc8e 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -98,11 +98,13 @@ static std::pair<OutputItT, IndexOutputItT> fst_baseline(InputItT begin, auto symbol_group = std::distance(std::cbegin(symbol_group_lut), symbol_group_it); // Output the translated symbols to the output tape - out_tape = std::copy(std::cbegin(translation_table[state][symbol_group]), - std::cend(translation_table[state][symbol_group]), - out_tape); - auto out_size = std::distance(std::cbegin(translation_table[state][symbol_group]), - std::cend(translation_table[state][symbol_group])); + out_tape = std::copy(std::cbegin(translation_table[state][symbol_group]), + std::cend(translation_table[state][symbol_group]), + out_tape); + + auto out_size = std::distance(std::cbegin(translation_table[state][symbol_group]), + std::cend(translation_table[state][symbol_group])); + out_index_tape = std::fill_n(out_index_tape, out_size, in_offset); // Transition the state of the finite-state machine From b37f71634104360e9be0e6c5b0ec69f3482ec975 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 21 Jul 2022 06:20:48 -0700 Subject: [PATCH 35/81] if constexprs and doxy on DFA helper --- cpp/src/io/fst/dispatch_dfa.cuh | 6 +++--- cpp/src/io/fst/in_reg_array.cuh | 4 ++-- cpp/src/io/fst/lookup_tables.cuh | 24 ++++++++++++++++++++++++ 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/cpp/src/io/fst/dispatch_dfa.cuh b/cpp/src/io/fst/dispatch_dfa.cuh index 2a5594c383d..cabbe863131 100644 --- a/cpp/src/io/fst/dispatch_dfa.cuh +++ b/cpp/src/io/fst/dispatch_dfa.cuh @@ -375,14 +375,14 @@ struct DispatchFSM : DeviceFSMPolicy { allocation_sizes[MEM_SCAN] = vector_scan_storage_bytes; // Bytes needed for tile status descriptors (fusing state-transition vector + DFA simulation) - if (SINGLE_PASS_STV) { + if constexpr (SINGLE_PASS_STV) { error = ScanTileStateT::AllocationSize(num_blocks, allocation_sizes[MEM_SINGLE_PASS_STV]); if (error != cudaSuccess) return error; } // Bytes needed for tile status descriptors (DFA simulation pass for output size computation + // output-generating pass) - if (IS_FST) { + if constexpr (IS_FST) { error = FstScanTileStateT::AllocationSize(num_blocks, allocation_sizes[MEM_FST_OFFSET]); if (error != cudaSuccess) return error; } @@ -404,7 +404,7 @@ struct DispatchFSM : DeviceFSMPolicy { // INITIALIZE SCAN TILE STATES COMPUTING TRANSDUCED OUTPUT OFFSETS //------------------------------------------------------------------------------ FstScanTileStateT fst_offset_tile_state; - if (IS_FST) { + if constexpr (IS_FST) { // Construct the tile status (aliases memory internally et al.) error = fst_offset_tile_state.Init( num_blocks, allocations[MEM_FST_OFFSET], allocation_sizes[MEM_FST_OFFSET]); diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index 87ea2d3325f..e26d494a557 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -62,7 +62,7 @@ class MultiFragmentInRegArray { /** * @brief Returns the \p num_bits bits starting at \p bit_start */ - CUDF_HOST_DEVICE uint32_t bfe(const uint32_t& data, uint32_t bit_start, uint32_t num_bits) const + CUDF_HOST_DEVICE [[nodiscard]] uint32_t bfe(const uint32_t& data, uint32_t bit_start, uint32_t num_bits) const { #if CUB_PTX_ARCH > 0 return cub::BFE(data, bit_start, num_bits); @@ -98,7 +98,7 @@ class MultiFragmentInRegArray { // ACCESSORS //------------------------------------------------------------------------------ public: - CUDF_HOST_DEVICE uint32_t Get(int32_t index) const + CUDF_HOST_DEVICE [[nodiscard]] uint32_t Get(int32_t index) const { uint32_t val = 0; diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index 341c3b7a51d..b364b2cc3d7 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -496,6 +496,30 @@ class Dfa { TranslationTableT::InitDeviceTranslationTable(translation_table_init, out_tt_vec, stream); } + /** + * @brief Dispatches the finite-state transducer algorithm to the GPU. + * + * @tparam SymbolT The atomic symbol type from the input tape + * @tparam TransducedOutItT Random-access output iterator to which the transduced output will be + * written + * @tparam TransducedIndexOutItT Random-access output iterator type to which the indexes of the + * symbols that caused some output to be written. + * @tparam TransducedCountOutItT A single-item output iterator type to which the total number of + * output symbols is written + * @tparam OffsetT A type large enough to index into either of both: (a) the input symbols and (b) + * the output symbols + * @param d_chars Pointer to the input string of symbols + * @param num_chars The total number of input symbols to process + * @param d_out_it Random-access output iterator to which the transduced output is + * written + * @param d_out_idx_it Random-access output iterator to which, the index i is written + * iff the i-th input symbol caused some output to be written + * @param d_num_transduced_out_it A single-item output iterator type to which the total number + * of output symbols is written + * @param seed_state The DFA's starting state. For streaming DFAs this corresponds to the + * "end-state" of the previous invocation of the algorithm. + * @param stream CUDA stream to launch kernels within. Default is the null-stream. + */ template <typename SymbolT, typename TransducedOutItT, typename TransducedIndexOutItT, From d42869a1d659c8a7d459b0a99e60596f83d0d36a Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 21 Jul 2022 07:56:31 -0700 Subject: [PATCH 36/81] minor documentation fix --- cpp/src/io/fst/device_dfa.cuh | 4 ++-- cpp/src/io/fst/lookup_tables.cuh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cpp/src/io/fst/device_dfa.cuh b/cpp/src/io/fst/device_dfa.cuh index 56afc32e643..7eeff27eef1 100644 --- a/cpp/src/io/fst/device_dfa.cuh +++ b/cpp/src/io/fst/device_dfa.cuh @@ -31,8 +31,8 @@ namespace cudf::io::fst { * @tparam SymbolItT Random-access input iterator type to symbols fed into the FST * @tparam TransducedOutItT Random-access output iterator to which the transduced output will be * written - * @tparam TransducedIndexOutItT Random-access output iterator type to which the indexes of the - * symbols that caused some output to be written. + * @tparam TransducedIndexOutItT Random-access output iterator type to which the input symbols' + * indexes are written. * @tparam TransducedCountOutItT A single-item output iterator type to which the total number of * output symbols is written * @tparam OffsetT A type large enough to index into either of both: (a) the input symbols and (b) diff --git a/cpp/src/io/fst/lookup_tables.cuh b/cpp/src/io/fst/lookup_tables.cuh index b364b2cc3d7..c5033868925 100644 --- a/cpp/src/io/fst/lookup_tables.cuh +++ b/cpp/src/io/fst/lookup_tables.cuh @@ -502,8 +502,8 @@ class Dfa { * @tparam SymbolT The atomic symbol type from the input tape * @tparam TransducedOutItT Random-access output iterator to which the transduced output will be * written - * @tparam TransducedIndexOutItT Random-access output iterator type to which the indexes of the - * symbols that caused some output to be written. + * @tparam TransducedIndexOutItT Random-access output iterator type to which the input symbols' + * indexes are written. * @tparam TransducedCountOutItT A single-item output iterator type to which the total number of * output symbols is written * @tparam OffsetT A type large enough to index into either of both: (a) the input symbols and (b) From 6c889f70e71e2cd750ca780d12812feff3015983 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 21 Jul 2022 09:52:37 -0700 Subject: [PATCH 37/81] replaces loop for comparing vectors with generic macro --- cpp/include/cudf_test/cudf_gtest.hpp | 9 +++++++++ cpp/src/io/fst/in_reg_array.cuh | 4 +++- cpp/tests/io/fst/fst_test.cu | 8 ++------ 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/cpp/include/cudf_test/cudf_gtest.hpp b/cpp/include/cudf_test/cudf_gtest.hpp index 6c62b0159ca..fb2680545d3 100644 --- a/cpp/include/cudf_test/cudf_gtest.hpp +++ b/cpp/include/cudf_test/cudf_gtest.hpp @@ -176,3 +176,12 @@ struct TypeList<Types<TYPES...>> { } catch (std::exception & e) { \ FAIL() << "statement:" << #statement << std::endl << "reason: " << e.what() << std::endl; \ } + +/** + * @brief test macro comparing for equality of \p lhs and and \p rhs for the first \p size elements. + */ +#define CUDF_TEST_EXPECT_VECTOR_EQUAL(lhs, rhs, size) \ + do { \ + for (decltype(size) i = 0; i < size; i++) \ + EXPECT_EQ(lhs[i], rhs[i]) << "Mismatch at index #" << i; \ + } while (0) diff --git a/cpp/src/io/fst/in_reg_array.cuh b/cpp/src/io/fst/in_reg_array.cuh index e26d494a557..0819deb6d97 100644 --- a/cpp/src/io/fst/in_reg_array.cuh +++ b/cpp/src/io/fst/in_reg_array.cuh @@ -62,7 +62,9 @@ class MultiFragmentInRegArray { /** * @brief Returns the \p num_bits bits starting at \p bit_start */ - CUDF_HOST_DEVICE [[nodiscard]] uint32_t bfe(const uint32_t& data, uint32_t bit_start, uint32_t num_bits) const + CUDF_HOST_DEVICE [[nodiscard]] uint32_t bfe(const uint32_t& data, + uint32_t bit_start, + uint32_t num_bits) const { #if CUB_PTX_ARCH > 0 return cub::BFE(data, bit_start, num_bits); diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index 40866d8bc8e..e198c804222 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -255,12 +255,8 @@ TEST_F(FstTest, GroundTruth) // Verify results ASSERT_EQ(output_gpu_size[0], output_cpu.size()); - for (std::size_t i = 0; i < output_cpu.size(); i++) { - EXPECT_EQ(output_gpu[i], output_cpu[i]) << "Mismatch at index #" << i; - } - for (std::size_t i = 0; i < output_cpu.size(); i++) { - EXPECT_EQ(out_indexes_gpu[i], out_index_cpu[i]) << "Mismatch at index #" << i; - } + CUDF_TEST_EXPECT_VECTOR_EQUAL(output_gpu, output_cpu, output_cpu.size()); + CUDF_TEST_EXPECT_VECTOR_EQUAL(out_indexes_gpu, out_index_cpu, output_cpu.size()); } CUDF_TEST_PROGRAM_MAIN() From 8a54c728aeccb01d1e6b6ee188901b3973986d55 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 21 Jul 2022 09:52:57 -0700 Subject: [PATCH 38/81] uses new vector comparison for logical stack test --- cpp/tests/io/fst/logical_stack_test.cu | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/cpp/tests/io/fst/logical_stack_test.cu b/cpp/tests/io/fst/logical_stack_test.cu index d76ff16f85d..dda737f005d 100644 --- a/cpp/tests/io/fst/logical_stack_test.cu +++ b/cpp/tests/io/fst/logical_stack_test.cu @@ -244,9 +244,7 @@ TEST_F(LogicalStackTest, GroundTruth) // Verify results ASSERT_EQ(string_size, top_of_stack_cpu.size()); ASSERT_EQ(top_of_stack_gpu.size(), top_of_stack_cpu.size()); - for (size_t i = 0; i < string_size && i < top_of_stack_cpu.size(); i++) { - ASSERT_EQ(top_of_stack_gpu.host_ptr()[i], top_of_stack_cpu[i]) << "Mismatch at index #" << i; - } + CUDF_TEST_EXPECT_VECTOR_EQUAL(top_of_stack_gpu.host_ptr(), top_of_stack_cpu, string_size); } CUDF_TEST_PROGRAM_MAIN() From cc1e135684deb1e9e04d14a0ab8692f02e6a89bd Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 31 Mar 2022 04:11:44 -0700 Subject: [PATCH 39/81] Added utility to debug print & instrumented code to use it --- cpp/src/io/fst/logical_stack.cuh | 62 ++++++++++++++++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/cpp/src/io/fst/logical_stack.cuh b/cpp/src/io/fst/logical_stack.cuh index 9502922a379..60a038a394c 100644 --- a/cpp/src/io/fst/logical_stack.cuh +++ b/cpp/src/io/fst/logical_stack.cuh @@ -216,6 +216,46 @@ struct RemapEmptyStack { StackOpT empty_stack_symbol; }; +/** + * @brief Function object to return only the key part from a KeyValueOp instance. + */ +struct KVOpToKey { + template <typename KeyT, typename ValueT> + constexpr CUDF_HOST_DEVICE KeyT operator()(KeyValueOp<KeyT, ValueT> const& kv_op) const + { + return kv_op.key; + } +}; + +/** + * @brief Function object to return only the value part from a KeyValueOp instance. + */ +struct KVOpToValue { + template <typename KeyT, typename ValueT> + constexpr CUDF_HOST_DEVICE ValueT operator()(KeyValueOp<KeyT, ValueT> const& kv_op) const + { + return kv_op.value; + } +}; + +/** + * @brief Retrieves an iterator that returns only the `key` part from a KeyValueOp iterator. + */ +template <typename KeyValueOpItT> +auto get_key_it(KeyValueOpItT it) +{ + return thrust::make_transform_iterator(it, KVOpToKey{}); +} + +/** + * @brief Retrieves an iterator that returns only the `value` part from a KeyValueOp iterator. + */ +template <typename KeyValueOpItT> +auto get_value_it(KeyValueOpItT it) +{ + return thrust::make_transform_iterator(it, KVOpToValue{}); +} + } // namespace detail /** @@ -401,6 +441,14 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, num_symbols_in, stream)); + // Dump info on stack operations: (stack level change + symbol) -> (absolute stack level + symbol) + test::print::print_array(num_symbols_in, + stream, + get_key_it(stack_symbols_in), + get_value_it(stack_symbols_in), + get_key_it(d_kv_operations.Current()), + get_value_it(d_kv_operations.Current())); + // Stable radix sort, sorting by stack level of the operations d_kv_operations_unsigned = cub::DoubleBuffer<StackOpUnsignedT>{ reinterpret_cast<StackOpUnsignedT*>(d_kv_operations.Current()), @@ -429,6 +477,15 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, num_symbols_in, stream)); + // Dump info on stack operations sorted by their stack level (i.e. stack level after applying + // operation) + test::print::print_array(num_symbols_in, + stream, + get_key_it(kv_ops_scan_in), + get_value_it(kv_ops_scan_in), + get_key_it(kv_ops_scan_out), + get_value_it(kv_ops_scan_out)); + // Fill the output tape with read-symbol thrust::fill(rmm::exec_policy(stream), thrust::device_ptr<StackSymbolT>{d_top_of_stack}, @@ -447,6 +504,11 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, d_symbol_positions_db.Current(), d_top_of_stack); + // Dump the output tape that has many yet-to-be-filled spots (i.e., all spots that were not given + // in the sparse representation) + test::print::print_array( + std::min(num_symbols_in, static_cast<decltype(num_symbols_in)>(10000)), stream, d_top_of_stack); + // We perform an exclusive scan in order to fill the items at the very left that may // be reading the empty stack before there's the first push occurrence in the sequence. // Also, we're interested in the top-of-the-stack symbol before the operation was applied. From 7dba1777ca1d8f9937ca76f5ee5010abcff8e4d2 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 31 Mar 2022 05:28:17 -0700 Subject: [PATCH 40/81] switched to using rmm also inside algorithm --- cpp/tests/io/fst/logical_stack_test.cu | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/tests/io/fst/logical_stack_test.cu b/cpp/tests/io/fst/logical_stack_test.cu index dda737f005d..9b93397fd3b 100644 --- a/cpp/tests/io/fst/logical_stack_test.cu +++ b/cpp/tests/io/fst/logical_stack_test.cu @@ -23,6 +23,7 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> #include <rmm/device_uvector.hpp> +#include <rmm/device_buffer.hpp> #include <cstdlib> #include <iostream> From ff7144ae98fd59f0031d340592593951cbaa98e5 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 4 Apr 2022 02:28:30 -0700 Subject: [PATCH 41/81] renaming key-value store op to stack_op --- cpp/src/io/fst/logical_stack.cuh | 45 ++++++++++++-------------------- 1 file changed, 17 insertions(+), 28 deletions(-) diff --git a/cpp/src/io/fst/logical_stack.cuh b/cpp/src/io/fst/logical_stack.cuh index 60a038a394c..30f28a48788 100644 --- a/cpp/src/io/fst/logical_stack.cuh +++ b/cpp/src/io/fst/logical_stack.cuh @@ -217,43 +217,32 @@ struct RemapEmptyStack { }; /** - * @brief Function object to return only the key part from a KeyValueOp instance. + * @brief Function object to return only the stack_level part from a StackOp instance. */ -struct KVOpToKey { - template <typename KeyT, typename ValueT> - constexpr CUDF_HOST_DEVICE KeyT operator()(KeyValueOp<KeyT, ValueT> const& kv_op) const - { - return kv_op.key; - } -}; - -/** - * @brief Function object to return only the value part from a KeyValueOp instance. - */ -struct KVOpToValue { - template <typename KeyT, typename ValueT> - constexpr CUDF_HOST_DEVICE ValueT operator()(KeyValueOp<KeyT, ValueT> const& kv_op) const +struct StackOpToStackLevel { + template <typename StackLevelT, typename ValueT> + constexpr CUDF_HOST_DEVICE StackLevelT operator()(StackOp<StackLevelT, ValueT> const& kv_op) const { - return kv_op.value; + return kv_op.stack_level; } }; /** - * @brief Retrieves an iterator that returns only the `key` part from a KeyValueOp iterator. + * @brief Retrieves an iterator that returns only the `stack_level` part from a StackOp iterator. */ -template <typename KeyValueOpItT> -auto get_key_it(KeyValueOpItT it) +template <typename StackOpItT> +auto get_stack_level_it(StackOpItT it) { - return thrust::make_transform_iterator(it, KVOpToKey{}); + return thrust::make_transform_iterator(it, StackOpToStackLevel{}); } /** - * @brief Retrieves an iterator that returns only the `value` part from a KeyValueOp iterator. + * @brief Retrieves an iterator that returns only the `value` part from a StackOp iterator. */ -template <typename KeyValueOpItT> -auto get_value_it(KeyValueOpItT it) +template <typename StackOpItT> +auto get_value_it(StackOpItT it) { - return thrust::make_transform_iterator(it, KVOpToValue{}); + return thrust::make_transform_iterator(it, StackOpToStackSymbol{}); } } // namespace detail @@ -444,9 +433,9 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, // Dump info on stack operations: (stack level change + symbol) -> (absolute stack level + symbol) test::print::print_array(num_symbols_in, stream, - get_key_it(stack_symbols_in), + get_stack_level_it(stack_symbols_in), get_value_it(stack_symbols_in), - get_key_it(d_kv_operations.Current()), + get_stack_level_it(d_kv_operations.Current()), get_value_it(d_kv_operations.Current())); // Stable radix sort, sorting by stack level of the operations @@ -481,9 +470,9 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, // operation) test::print::print_array(num_symbols_in, stream, - get_key_it(kv_ops_scan_in), + get_stack_level_it(kv_ops_scan_in), get_value_it(kv_ops_scan_in), - get_key_it(kv_ops_scan_out), + get_stack_level_it(kv_ops_scan_out), get_value_it(kv_ops_scan_out)); // Fill the output tape with read-symbol From 61a76b7f495cf716d91bad864879f6e1f16f578d Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 4 Apr 2022 07:35:33 -0700 Subject: [PATCH 42/81] device_span --- cpp/tests/io/fst/logical_stack_test.cu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/tests/io/fst/logical_stack_test.cu b/cpp/tests/io/fst/logical_stack_test.cu index 9b93397fd3b..831e9140989 100644 --- a/cpp/tests/io/fst/logical_stack_test.cu +++ b/cpp/tests/io/fst/logical_stack_test.cu @@ -22,8 +22,8 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> -#include <rmm/device_uvector.hpp> #include <rmm/device_buffer.hpp> +#include <rmm/device_uvector.hpp> #include <cstdlib> #include <iostream> From c28e327c7e33112a499eeed4d0c844655bf22c15 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Tue, 12 Apr 2022 22:55:00 -0700 Subject: [PATCH 43/81] minor style changes addressing review comments --- cpp/src/io/fst/logical_stack.cuh | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/cpp/src/io/fst/logical_stack.cuh b/cpp/src/io/fst/logical_stack.cuh index 30f28a48788..5149ade7c1c 100644 --- a/cpp/src/io/fst/logical_stack.cuh +++ b/cpp/src/io/fst/logical_stack.cuh @@ -231,7 +231,7 @@ struct StackOpToStackLevel { * @brief Retrieves an iterator that returns only the `stack_level` part from a StackOp iterator. */ template <typename StackOpItT> -auto get_stack_level_it(StackOpItT it) +auto get_stack_level_iterator(StackOpItT it) { return thrust::make_transform_iterator(it, StackOpToStackLevel{}); } @@ -240,7 +240,7 @@ auto get_stack_level_it(StackOpItT it) * @brief Retrieves an iterator that returns only the `value` part from a StackOp iterator. */ template <typename StackOpItT> -auto get_value_it(StackOpItT it) +auto get_value_iterator(StackOpItT it) { return thrust::make_transform_iterator(it, StackOpToStackSymbol{}); } @@ -433,10 +433,10 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, // Dump info on stack operations: (stack level change + symbol) -> (absolute stack level + symbol) test::print::print_array(num_symbols_in, stream, - get_stack_level_it(stack_symbols_in), - get_value_it(stack_symbols_in), - get_stack_level_it(d_kv_operations.Current()), - get_value_it(d_kv_operations.Current())); + get_stack_level_iterator(stack_symbols_in), + get_value_iterator(stack_symbols_in), + get_stack_level_iterator(d_kv_operations.Current()), + get_value_iterator(d_kv_operations.Current())); // Stable radix sort, sorting by stack level of the operations d_kv_operations_unsigned = cub::DoubleBuffer<StackOpUnsignedT>{ @@ -470,10 +470,10 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, // operation) test::print::print_array(num_symbols_in, stream, - get_stack_level_it(kv_ops_scan_in), - get_value_it(kv_ops_scan_in), - get_stack_level_it(kv_ops_scan_out), - get_value_it(kv_ops_scan_out)); + get_stack_level_iterator(kv_ops_scan_in), + get_value_iterator(kv_ops_scan_in), + get_stack_level_iterator(kv_ops_scan_out), + get_value_iterator(kv_ops_scan_out)); // Fill the output tape with read-symbol thrust::fill(rmm::exec_policy(stream), From a2f27ae293b9e0653e4bc3ebcedbc97aa3808b73 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 11 Apr 2022 12:17:55 -0700 Subject: [PATCH 44/81] squashed with bracket/brace test --- cpp/src/io/fst/symbol_lut.cuh | 182 ++++++++++++++++++++++++ cpp/src/io/fst/transition_table.cuh | 149 ++++++++++++++++++++ cpp/src/io/fst/translation_table.cuh | 200 +++++++++++++++++++++++++++ 3 files changed, 531 insertions(+) create mode 100644 cpp/src/io/fst/symbol_lut.cuh create mode 100644 cpp/src/io/fst/transition_table.cuh create mode 100644 cpp/src/io/fst/translation_table.cuh diff --git a/cpp/src/io/fst/symbol_lut.cuh b/cpp/src/io/fst/symbol_lut.cuh new file mode 100644 index 00000000000..08d5f4db58d --- /dev/null +++ b/cpp/src/io/fst/symbol_lut.cuh @@ -0,0 +1,182 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <cub/cub.cuh> + +#include <algorithm> +#include <cstdint> +#include <vector> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { +/** + * @brief Class template that can be plugged into the finite-state machine to look up the symbol + * group index for a given symbol. Class template does not support multi-symbol lookups (i.e., no + * look-ahead). + * + * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id + */ +template <typename SymbolT> +struct SingleSymbolSmemLUT { + //------------------------------------------------------------------------------ + // DEFAULT TYPEDEFS + //------------------------------------------------------------------------------ + // Type used for representing a symbol group id (i.e., what we return for a given symbol) + using SymbolGroupIdT = uint8_t; + + //------------------------------------------------------------------------------ + // DERIVED CONFIGURATIONS + //------------------------------------------------------------------------------ + /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) + static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); + + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + + struct _TempStorage { + // d_match_meta_data[symbol] -> symbol group index + SymbolGroupIdT match_meta_data[NUM_ENTRIES_PER_LUT]; + }; + + struct KernelParameter { + // d_match_meta_data[min(symbol,num_valid_entries)] -> symbol group index + SymbolGroupIdT num_valid_entries; + + // d_match_meta_data[symbol] -> symbol group index + SymbolGroupIdT* d_match_meta_data; + }; + + struct TempStorage : cub::Uninitialized<_TempStorage> { + }; + + //------------------------------------------------------------------------------ + // HELPER METHODS + //------------------------------------------------------------------------------ + /** + * @brief + * + * @param[in] d_temp_storage Device-side temporary storage that can be used to store the lookup + * table. If no storage is provided it will return the temporary storage requirements in \p + * d_temp_storage_bytes. + * @param[in,out] d_temp_storage_bytes Amount of device-side temporary storage that can be used in + * the number of bytes + * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols + * (characters!) that correspond to the i-th symbol group index + * @param[out] kernel_param The kernel parameter object to be initialized with the given mapping + * of symbols to symbol group ids. + * @param[in] stream The stream that shall be used to cudaMemcpyAsync the lookup table + * @return + */ + template <typename SymbolGroupItT> + __host__ __forceinline__ static cudaError_t PrepareLUT(void* d_temp_storage, + size_t& d_temp_storage_bytes, + SymbolGroupItT const& symbol_strings, + KernelParameter& kernel_param, + cudaStream_t stream = 0) + { + // The symbol group index to be returned if none of the given symbols match + SymbolGroupIdT no_match_id = symbol_strings.size(); + + std::vector<SymbolGroupIdT> lut(NUM_ENTRIES_PER_LUT); + SymbolGroupIdT max_base_match_val = 0; + + // Initialize all entries: by default we return the no-match-id + for (uint32_t i = 0; i < NUM_ENTRIES_PER_LUT; ++i) { + lut[i] = no_match_id; + } + + // Set up lookup table + uint32_t sg_id = 0; + for (auto const& sg_symbols : symbol_strings) { + for (auto const& sg_symbol : sg_symbols) { + max_base_match_val = std::max(max_base_match_val, static_cast<SymbolGroupIdT>(sg_symbol)); + lut[sg_symbol] = sg_id; + } + sg_id++; + } + + // Initialize the out-of-bounds lookup: d_match_meta_data[max_base_match_val+1] -> no_match_id + lut[max_base_match_val + 1] = no_match_id; + + // Alias memory / return memory requiremenets + kernel_param.num_valid_entries = max_base_match_val + 2; + if (d_temp_storage) { + cudaError_t error = cudaMemcpyAsync(d_temp_storage, + lut.data(), + kernel_param.num_valid_entries * sizeof(SymbolGroupIdT), + cudaMemcpyHostToDevice, + stream); + + kernel_param.d_match_meta_data = reinterpret_cast<SymbolGroupIdT*>(d_temp_storage); + return error; + } else { + d_temp_storage_bytes = kernel_param.num_valid_entries * sizeof(SymbolGroupIdT); + return cudaSuccess; + } + + return cudaSuccess; + } + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + SymbolGroupIdT num_valid_entries; + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + __host__ __device__ __forceinline__ SingleSymbolSmemLUT(KernelParameter const& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) + { + // GPU-side init +#if CUB_PTX_ARCH > 0 + for (int32_t i = threadIdx.x; i < kernel_param.num_valid_entries; i += blockDim.x) { + this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + } + __syncthreads(); + +#else + // CPU-side init + for (std::size_t i = 0; i < kernel_param.num_luts; i++) { + this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + } +#endif + } + + __host__ __device__ __forceinline__ int32_t operator()(SymbolT const symbol) const + { + // Look up the symbol group for given symbol + return temp_storage.match_meta_data[min(symbol, num_valid_entries - 1)]; + } +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/transition_table.cuh b/cpp/src/io/fst/transition_table.cuh new file mode 100644 index 00000000000..97fef03d8af --- /dev/null +++ b/cpp/src/io/fst/transition_table.cuh @@ -0,0 +1,149 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <cub/cub.cuh> + +#include <cstdint> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +template <int MAX_NUM_SYMBOLS, int MAX_NUM_STATES> +struct TransitionTable { + //------------------------------------------------------------------------------ + // DEFAULT TYPEDEFS + //------------------------------------------------------------------------------ + using ItemT = char; + + struct TransitionVectorWrapper { + const ItemT* data; + + __host__ __device__ TransitionVectorWrapper(const ItemT* data) : data(data) {} + + __host__ __device__ __forceinline__ uint32_t Get(int32_t index) const { return data[index]; } + }; + + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + using TransitionVectorT = TransitionVectorWrapper; + + struct _TempStorage { + // + ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; + }; + + struct TempStorage : cub::Uninitialized<_TempStorage> { + }; + + struct KernelParameter { + ItemT* transitions; + }; + + using LoadAliasT = std::uint32_t; + + static constexpr std::size_t NUM_AUX_MEM_BYTES = + CUB_QUOTIENT_CEILING(MAX_NUM_STATES * MAX_NUM_SYMBOLS * sizeof(ItemT), sizeof(LoadAliasT)) * + sizeof(LoadAliasT); + + //------------------------------------------------------------------------------ + // HELPER METHODS + //------------------------------------------------------------------------------ + __host__ static cudaError_t CreateTransitionTable( + void* d_temp_storage, + size_t& temp_storage_bytes, + const std::vector<std::vector<int>>& trans_table, + KernelParameter& kernel_param, + cudaStream_t stream = 0) + { + if (!d_temp_storage) { + temp_storage_bytes = NUM_AUX_MEM_BYTES; + return cudaSuccess; + } + + // trans_vectors[symbol][state] -> new_state + ItemT trans_vectors[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; + + // trans_table[state][symbol] -> new state + for (std::size_t state = 0; state < trans_table.size(); ++state) { + for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { + trans_vectors[symbol * MAX_NUM_STATES + state] = trans_table[state][symbol]; + } + } + + kernel_param.transitions = static_cast<ItemT*>(d_temp_storage); + + // Copy transition table to device + return cudaMemcpyAsync( + d_temp_storage, trans_vectors, NUM_AUX_MEM_BYTES, cudaMemcpyHostToDevice, stream); + } + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __host__ __device__ __forceinline__ TransitionTable(const KernelParameter& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()) + { +#if CUB_PTX_ARCH > 0 + for (int i = threadIdx.x; i < CUB_QUOTIENT_CEILING(NUM_AUX_MEM_BYTES, sizeof(LoadAliasT)); + i += blockDim.x) { + reinterpret_cast<LoadAliasT*>(this->temp_storage.transitions)[i] = + reinterpret_cast<LoadAliasT*>(kernel_param.transitions)[i]; + } + __syncthreads(); +#else + for (int i = 0; i < kernel_param.num_luts; i++) { + this->temp_storage.transitions[i] = kernel_param.transitions[i]; + } +#endif + } + + /** + * @brief Returns a random-access iterator to lookup all the state transitions for one specific + * symbol from an arbitrary old_state, i.e., it[old_state] -> new_state. + * + * @param state_id The DFA's current state index from which we'll transition + * @param match_id The symbol group id of the symbol that we just read in + * @return + */ + template <typename StateIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ int32_t operator()(StateIndexT state_id, + SymbolIndexT match_id) const + { + return temp_storage.transitions[match_id * MAX_NUM_STATES + state_id]; + } +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/fst/translation_table.cuh b/cpp/src/io/fst/translation_table.cuh new file mode 100644 index 00000000000..bfbfd41e3f0 --- /dev/null +++ b/cpp/src/io/fst/translation_table.cuh @@ -0,0 +1,200 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 "in_reg_array.cuh" + +#include <cub/cub.cuh> + +#include <cstdint> + +namespace cudf { +namespace io { +namespace fst { +namespace detail { + +/** + * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols to + * output + * + * @tparam OutSymbolT The symbol type being returned + * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols + * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition + * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support + * @tparam MAX_TABLE_SIZE The maximum number of items in the lookup table of output symbols + */ +template <typename OutSymbolT, + typename OutSymbolOffsetT, + int32_t MAX_NUM_SYMBOLS, + int32_t MAX_NUM_STATES, + int32_t MAX_TABLE_SIZE = (MAX_NUM_SYMBOLS * MAX_NUM_STATES)> +struct TransducerLookupTable { + //------------------------------------------------------------------------------ + // TYPEDEFS + //------------------------------------------------------------------------------ + struct _TempStorage { + OutSymbolOffsetT out_offset[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; + OutSymbolT out_symbols[MAX_TABLE_SIZE]; + }; + + struct TempStorage : cub::Uninitialized<_TempStorage> { + }; + + struct KernelParameter { + OutSymbolOffsetT* d_trans_offsets; + OutSymbolT* d_out_symbols; + }; + + //------------------------------------------------------------------------------ + // HELPER METHODS + //------------------------------------------------------------------------------ + __host__ static cudaError_t CreateTransitionTable( + void* d_temp_storage, + size_t& temp_storage_bytes, + const std::vector<std::vector<std::vector<OutSymbolT>>>& trans_table, + KernelParameter& kernel_param, + cudaStream_t stream = 0) + { + enum { MEM_OFFSETS = 0, MEM_OUT_SYMBOLS, NUM_ALLOCATIONS }; + + size_t allocation_sizes[NUM_ALLOCATIONS] = {}; + void* allocations[NUM_ALLOCATIONS] = {}; + allocation_sizes[MEM_OFFSETS] = + (MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1) * sizeof(OutSymbolOffsetT); + allocation_sizes[MEM_OUT_SYMBOLS] = MAX_TABLE_SIZE * sizeof(OutSymbolT); + + // Alias the temporary allocations from the single storage blob (or compute the necessary size + // of the blob) + cudaError_t error = + cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); + if (error) return error; + + // Return if the caller is simply requesting the size of the storage allocation + if (d_temp_storage == nullptr) return cudaSuccess; + + std::vector<OutSymbolT> out_symbols; + out_symbols.reserve(MAX_TABLE_SIZE); + std::vector<OutSymbolOffsetT> out_symbol_offsets; + out_symbol_offsets.reserve(MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1); + out_symbol_offsets.push_back(0); + + int st = 0; + // Iterate over the states in the transition table + for (auto const& state_trans : trans_table) { + uint32_t num_added = 0; + // Iterate over the symbols in the transition table + for (auto const& symbol_out : state_trans) { + // Insert the output symbols for this specific (state, symbol) transition + out_symbols.insert(std::end(out_symbols), std::begin(symbol_out), std::end(symbol_out)); + out_symbol_offsets.push_back(out_symbols.size()); + num_added++; + } + st++; + + // Copy the last offset for all symbols (to guarantee a proper lookup for omitted symbols of + // this state) + if (MAX_NUM_SYMBOLS > num_added) { + int32_t count = MAX_NUM_SYMBOLS - num_added; + auto begin_it = std::prev(std::end(out_symbol_offsets)); + std::copy(begin_it, begin_it + count, std::back_inserter(out_symbol_offsets)); + } + } + + // Check whether runtime-provided table size exceeds the compile-time given max. table size + if (out_symbols.size() > MAX_TABLE_SIZE) { return cudaErrorInvalidValue; } + + kernel_param.d_trans_offsets = static_cast<OutSymbolOffsetT*>(allocations[MEM_OFFSETS]); + kernel_param.d_out_symbols = static_cast<OutSymbolT*>(allocations[MEM_OUT_SYMBOLS]); + + // Copy out symbols + error = cudaMemcpyAsync(kernel_param.d_trans_offsets, + out_symbol_offsets.data(), + out_symbol_offsets.size() * sizeof(out_symbol_offsets[0]), + cudaMemcpyHostToDevice, + stream); + if (error) { return error; } + + // Copy offsets into output symbols + return cudaMemcpyAsync(kernel_param.d_out_symbols, + out_symbols.data(), + out_symbols.size() * sizeof(out_symbols[0]), + cudaMemcpyHostToDevice, + stream); + } + + //------------------------------------------------------------------------------ + // MEMBER VARIABLES + //------------------------------------------------------------------------------ + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + return private_storage; + } + + //------------------------------------------------------------------------------ + // CONSTRUCTOR + //------------------------------------------------------------------------------ + __host__ __device__ __forceinline__ TransducerLookupTable(const KernelParameter& kernel_param, + TempStorage& temp_storage) + : temp_storage(temp_storage.Alias()) + { + constexpr uint32_t num_offsets = MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1; +#if CUB_PTX_ARCH > 0 + for (int i = threadIdx.x; i < num_offsets; i += blockDim.x) { + this->temp_storage.out_offset[i] = kernel_param.d_trans_offsets[i]; + } + // Make sure all threads in the block can read out_symbol_offsets[num_offsets - 1] from shared + // memory + __syncthreads(); + for (int i = threadIdx.x; i < this->temp_storage.out_offset[num_offsets - 1]; i += blockDim.x) { + this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; + } + __syncthreads(); +#else + for (int i = 0; i < num_offsets; i++) { + this->temp_storage.out_symbol_offsets[i] = kernel_param.d_trans_offsets[i]; + } + for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { + this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; + } +#endif + } + + template <typename StateIndexT, typename SymbolIndexT, typename RelativeOffsetT> + __host__ __device__ __forceinline__ OutSymbolT operator()(StateIndexT state_id, + SymbolIndexT match_id, + RelativeOffsetT relative_offset) const + { + auto offset = temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id] + relative_offset; + return temp_storage.out_symbols[offset]; + } + + template <typename StateIndexT, typename SymbolIndexT> + __host__ __device__ __forceinline__ OutSymbolOffsetT operator()(StateIndexT state_id, + SymbolIndexT match_id) const + { + return temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id + 1] - + temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id]; + } +}; + +} // namespace detail +} // namespace fst +} // namespace io +} // namespace cudf From fe4762df7ce0d60db4337fe60ef09376006552a6 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 25 Apr 2022 09:59:37 -0700 Subject: [PATCH 45/81] refactored lookup tables --- cpp/src/io/fst/symbol_lut.cuh | 94 ++++++++------------ cpp/src/io/fst/transition_table.cuh | 109 ++++++++---------------- cpp/src/io/fst/translation_table.cuh | 123 +++++++++++---------------- 3 files changed, 119 insertions(+), 207 deletions(-) diff --git a/cpp/src/io/fst/symbol_lut.cuh b/cpp/src/io/fst/symbol_lut.cuh index 08d5f4db58d..abf71a7fbea 100644 --- a/cpp/src/io/fst/symbol_lut.cuh +++ b/cpp/src/io/fst/symbol_lut.cuh @@ -16,6 +16,9 @@ #pragma once +#include <cudf/types.hpp> +#include <io/utilities/hostdevice_vector.hpp> + #include <cub/cub.cuh> #include <algorithm> @@ -34,38 +37,29 @@ namespace detail { * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id */ template <typename SymbolT> -struct SingleSymbolSmemLUT { - //------------------------------------------------------------------------------ - // DEFAULT TYPEDEFS - //------------------------------------------------------------------------------ +class SingleSymbolSmemLUT { + private: // Type used for representing a symbol group id (i.e., what we return for a given symbol) using SymbolGroupIdT = uint8_t; - //------------------------------------------------------------------------------ - // DERIVED CONFIGURATIONS - //------------------------------------------------------------------------------ /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); - //------------------------------------------------------------------------------ - // TYPEDEFS - //------------------------------------------------------------------------------ - struct _TempStorage { - // d_match_meta_data[symbol] -> symbol group index - SymbolGroupIdT match_meta_data[NUM_ENTRIES_PER_LUT]; + // sym_to_sgid[symbol] -> symbol group index + SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; }; + public: struct KernelParameter { - // d_match_meta_data[min(symbol,num_valid_entries)] -> symbol group index - SymbolGroupIdT num_valid_entries; + // sym_to_sgid[min(symbol,num_valid_entries)] -> symbol group index + SymbolT num_valid_entries; - // d_match_meta_data[symbol] -> symbol group index - SymbolGroupIdT* d_match_meta_data; + // sym_to_sgid[symbol] -> symbol group index + SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; }; - struct TempStorage : cub::Uninitialized<_TempStorage> { - }; + using TempStorage = cub::Uninitialized<_TempStorage>; //------------------------------------------------------------------------------ // HELPER METHODS @@ -73,66 +67,48 @@ struct SingleSymbolSmemLUT { /** * @brief * - * @param[in] d_temp_storage Device-side temporary storage that can be used to store the lookup - * table. If no storage is provided it will return the temporary storage requirements in \p - * d_temp_storage_bytes. - * @param[in,out] d_temp_storage_bytes Amount of device-side temporary storage that can be used in - * the number of bytes + * @param[out] sgid_init A hostdevice_vector that will be populated * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols * (characters!) that correspond to the i-th symbol group index - * @param[out] kernel_param The kernel parameter object to be initialized with the given mapping - * of symbols to symbol group ids. * @param[in] stream The stream that shall be used to cudaMemcpyAsync the lookup table * @return */ template <typename SymbolGroupItT> - __host__ __forceinline__ static cudaError_t PrepareLUT(void* d_temp_storage, - size_t& d_temp_storage_bytes, - SymbolGroupItT const& symbol_strings, - KernelParameter& kernel_param, - cudaStream_t stream = 0) + static void InitDeviceSymbolGroupIdLut(hostdevice_vector<KernelParameter>& sgid_init, + SymbolGroupItT const& symbol_strings, + rmm::cuda_stream_view stream) { // The symbol group index to be returned if none of the given symbols match SymbolGroupIdT no_match_id = symbol_strings.size(); - std::vector<SymbolGroupIdT> lut(NUM_ENTRIES_PER_LUT); + // The symbol with the largest value that is mapped to a symbol group id SymbolGroupIdT max_base_match_val = 0; // Initialize all entries: by default we return the no-match-id - for (uint32_t i = 0; i < NUM_ENTRIES_PER_LUT; ++i) { - lut[i] = no_match_id; - } + std::fill(&sgid_init.host_ptr()->sym_to_sgid[0], + &sgid_init.host_ptr()->sym_to_sgid[NUM_ENTRIES_PER_LUT], + no_match_id); // Set up lookup table uint32_t sg_id = 0; + // Iterate over the symbol groups for (auto const& sg_symbols : symbol_strings) { + // Iterate over all symbols that belong to the current symbol group for (auto const& sg_symbol : sg_symbols) { max_base_match_val = std::max(max_base_match_val, static_cast<SymbolGroupIdT>(sg_symbol)); - lut[sg_symbol] = sg_id; + sgid_init.host_ptr()->sym_to_sgid[static_cast<int32_t>(sg_symbol)] = sg_id; } sg_id++; } - // Initialize the out-of-bounds lookup: d_match_meta_data[max_base_match_val+1] -> no_match_id - lut[max_base_match_val + 1] = no_match_id; + // Initialize the out-of-bounds lookup: sym_to_sgid[max_base_match_val+1] -> no_match_id + sgid_init.host_ptr()->sym_to_sgid[max_base_match_val + 1] = no_match_id; // Alias memory / return memory requiremenets - kernel_param.num_valid_entries = max_base_match_val + 2; - if (d_temp_storage) { - cudaError_t error = cudaMemcpyAsync(d_temp_storage, - lut.data(), - kernel_param.num_valid_entries * sizeof(SymbolGroupIdT), - cudaMemcpyHostToDevice, - stream); - - kernel_param.d_match_meta_data = reinterpret_cast<SymbolGroupIdT*>(d_temp_storage); - return error; - } else { - d_temp_storage_bytes = kernel_param.num_valid_entries * sizeof(SymbolGroupIdT); - return cudaSuccess; - } + // TODO I think this could be +1? + sgid_init.host_ptr()->num_valid_entries = max_base_match_val + 2; - return cudaSuccess; + sgid_init.host_to_device(stream); } //------------------------------------------------------------------------------ @@ -150,29 +126,29 @@ struct SingleSymbolSmemLUT { return private_storage; } - __host__ __device__ __forceinline__ SingleSymbolSmemLUT(KernelParameter const& kernel_param, - TempStorage& temp_storage) + constexpr CUDF_HOST_DEVICE SingleSymbolSmemLUT(KernelParameter const& kernel_param, + TempStorage& temp_storage) : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) { // GPU-side init #if CUB_PTX_ARCH > 0 for (int32_t i = threadIdx.x; i < kernel_param.num_valid_entries; i += blockDim.x) { - this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; } __syncthreads(); #else // CPU-side init for (std::size_t i = 0; i < kernel_param.num_luts; i++) { - this->temp_storage.match_meta_data[i] = kernel_param.d_match_meta_data[i]; + this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; } #endif } - __host__ __device__ __forceinline__ int32_t operator()(SymbolT const symbol) const + constexpr CUDF_HOST_DEVICE int32_t operator()(SymbolT const symbol) const { // Look up the symbol group for given symbol - return temp_storage.match_meta_data[min(symbol, num_valid_entries - 1)]; + return temp_storage.sym_to_sgid[min(symbol, num_valid_entries - 1)]; } }; diff --git a/cpp/src/io/fst/transition_table.cuh b/cpp/src/io/fst/transition_table.cuh index 97fef03d8af..5eccb926974 100644 --- a/cpp/src/io/fst/transition_table.cuh +++ b/cpp/src/io/fst/transition_table.cuh @@ -16,6 +16,10 @@ #pragma once +#include <cudf/types.hpp> +#include <cudf/utilities/error.hpp> +#include <io/utilities/hostdevice_vector.hpp> + #include <cub/cub.cuh> #include <cstdint> @@ -25,103 +29,50 @@ namespace io { namespace fst { namespace detail { -template <int MAX_NUM_SYMBOLS, int MAX_NUM_STATES> -struct TransitionTable { - //------------------------------------------------------------------------------ - // DEFAULT TYPEDEFS - //------------------------------------------------------------------------------ +template <int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES> +class TransitionTable { + private: + // Type used using ItemT = char; - struct TransitionVectorWrapper { - const ItemT* data; - - __host__ __device__ TransitionVectorWrapper(const ItemT* data) : data(data) {} - - __host__ __device__ __forceinline__ uint32_t Get(int32_t index) const { return data[index]; } - }; - - //------------------------------------------------------------------------------ - // TYPEDEFS - //------------------------------------------------------------------------------ - using TransitionVectorT = TransitionVectorWrapper; - struct _TempStorage { - // ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; }; - struct TempStorage : cub::Uninitialized<_TempStorage> { - }; + public: + using TempStorage = cub::Uninitialized<_TempStorage>; struct KernelParameter { - ItemT* transitions; + ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; }; - using LoadAliasT = std::uint32_t; - - static constexpr std::size_t NUM_AUX_MEM_BYTES = - CUB_QUOTIENT_CEILING(MAX_NUM_STATES * MAX_NUM_SYMBOLS * sizeof(ItemT), sizeof(LoadAliasT)) * - sizeof(LoadAliasT); - - //------------------------------------------------------------------------------ - // HELPER METHODS - //------------------------------------------------------------------------------ - __host__ static cudaError_t CreateTransitionTable( - void* d_temp_storage, - size_t& temp_storage_bytes, - const std::vector<std::vector<int>>& trans_table, - KernelParameter& kernel_param, - cudaStream_t stream = 0) + static void InitDeviceTransitionTable(hostdevice_vector<KernelParameter>& transition_table_init, + const std::vector<std::vector<int>>& trans_table, + rmm::cuda_stream_view stream) { - if (!d_temp_storage) { - temp_storage_bytes = NUM_AUX_MEM_BYTES; - return cudaSuccess; - } - - // trans_vectors[symbol][state] -> new_state - ItemT trans_vectors[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; - // trans_table[state][symbol] -> new state for (std::size_t state = 0; state < trans_table.size(); ++state) { for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { - trans_vectors[symbol * MAX_NUM_STATES + state] = trans_table[state][symbol]; + transition_table_init.host_ptr()->transitions[symbol * MAX_NUM_STATES + state] = + trans_table[state][symbol]; } } - kernel_param.transitions = static_cast<ItemT*>(d_temp_storage); - // Copy transition table to device - return cudaMemcpyAsync( - d_temp_storage, trans_vectors, NUM_AUX_MEM_BYTES, cudaMemcpyHostToDevice, stream); + transition_table_init.host_to_device(stream); } - //------------------------------------------------------------------------------ - // MEMBER VARIABLES - //------------------------------------------------------------------------------ - _TempStorage& temp_storage; - - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - return private_storage; - } - - //------------------------------------------------------------------------------ - // CONSTRUCTOR - //------------------------------------------------------------------------------ - __host__ __device__ __forceinline__ TransitionTable(const KernelParameter& kernel_param, - TempStorage& temp_storage) + constexpr CUDF_HOST_DEVICE TransitionTable(const KernelParameter& kernel_param, + TempStorage& temp_storage) : temp_storage(temp_storage.Alias()) { #if CUB_PTX_ARCH > 0 - for (int i = threadIdx.x; i < CUB_QUOTIENT_CEILING(NUM_AUX_MEM_BYTES, sizeof(LoadAliasT)); - i += blockDim.x) { - reinterpret_cast<LoadAliasT*>(this->temp_storage.transitions)[i] = - reinterpret_cast<LoadAliasT*>(kernel_param.transitions)[i]; + for (int i = threadIdx.x; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i += blockDim.x) { + this->temp_storage.transitions[i] = kernel_param.transitions[i]; } __syncthreads(); #else - for (int i = 0; i < kernel_param.num_luts; i++) { + for (int i = 0; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i++) { this->temp_storage.transitions[i] = kernel_param.transitions[i]; } #endif @@ -136,11 +87,21 @@ struct TransitionTable { * @return */ template <typename StateIndexT, typename SymbolIndexT> - __host__ __device__ __forceinline__ int32_t operator()(StateIndexT state_id, - SymbolIndexT match_id) const + constexpr CUDF_HOST_DEVICE int32_t operator()(StateIndexT const state_id, + SymbolIndexT const match_id) const { return temp_storage.transitions[match_id * MAX_NUM_STATES + state_id]; - } + } + + private: + _TempStorage& temp_storage; + + __device__ __forceinline__ _TempStorage& PrivateStorage() + { + __shared__ _TempStorage private_storage; + + return private_storage; + } }; } // namespace detail diff --git a/cpp/src/io/fst/translation_table.cuh b/cpp/src/io/fst/translation_table.cuh index bfbfd41e3f0..89da994606c 100644 --- a/cpp/src/io/fst/translation_table.cuh +++ b/cpp/src/io/fst/translation_table.cuh @@ -16,7 +16,12 @@ #pragma once -#include "in_reg_array.cuh" +#include <algorithm> +#include <cudf/types.hpp> +#include <cudf/utilities/error.hpp> +#include <io/utilities/hostdevice_vector.hpp> + +#include "rmm/device_uvector.hpp" #include <cub/cub.cuh> @@ -28,10 +33,10 @@ namespace fst { namespace detail { /** - * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols to - * output + * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols + * that the finite-state transducer is supposed to output for each transition * - * @tparam OutSymbolT The symbol type being returned + * @tparam OutSymbolT The symbol type being output * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support @@ -42,57 +47,35 @@ template <typename OutSymbolT, int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES, int32_t MAX_TABLE_SIZE = (MAX_NUM_SYMBOLS * MAX_NUM_STATES)> -struct TransducerLookupTable { - //------------------------------------------------------------------------------ - // TYPEDEFS - //------------------------------------------------------------------------------ +class TransducerLookupTable { + private: struct _TempStorage { OutSymbolOffsetT out_offset[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; OutSymbolT out_symbols[MAX_TABLE_SIZE]; }; - struct TempStorage : cub::Uninitialized<_TempStorage> { - }; + public: + using TempStorage = cub::Uninitialized<_TempStorage>; struct KernelParameter { - OutSymbolOffsetT* d_trans_offsets; - OutSymbolT* d_out_symbols; + OutSymbolOffsetT d_out_offsets[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; + OutSymbolT d_out_symbols[MAX_TABLE_SIZE]; }; - //------------------------------------------------------------------------------ - // HELPER METHODS - //------------------------------------------------------------------------------ - __host__ static cudaError_t CreateTransitionTable( - void* d_temp_storage, - size_t& temp_storage_bytes, - const std::vector<std::vector<std::vector<OutSymbolT>>>& trans_table, - KernelParameter& kernel_param, - cudaStream_t stream = 0) + /** + * @brief Initializes the translation table (both the host and device parts) + */ + static void InitDeviceTranslationTable( + hostdevice_vector<KernelParameter>& translation_table_init, + std::vector<std::vector<std::vector<OutSymbolT>>> const& trans_table, + rmm::cuda_stream_view stream) { - enum { MEM_OFFSETS = 0, MEM_OUT_SYMBOLS, NUM_ALLOCATIONS }; - - size_t allocation_sizes[NUM_ALLOCATIONS] = {}; - void* allocations[NUM_ALLOCATIONS] = {}; - allocation_sizes[MEM_OFFSETS] = - (MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1) * sizeof(OutSymbolOffsetT); - allocation_sizes[MEM_OUT_SYMBOLS] = MAX_TABLE_SIZE * sizeof(OutSymbolT); - - // Alias the temporary allocations from the single storage blob (or compute the necessary size - // of the blob) - cudaError_t error = - cub::AliasTemporaries(d_temp_storage, temp_storage_bytes, allocations, allocation_sizes); - if (error) return error; - - // Return if the caller is simply requesting the size of the storage allocation - if (d_temp_storage == nullptr) return cudaSuccess; - std::vector<OutSymbolT> out_symbols; out_symbols.reserve(MAX_TABLE_SIZE); std::vector<OutSymbolOffsetT> out_symbol_offsets; out_symbol_offsets.reserve(MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1); out_symbol_offsets.push_back(0); - int st = 0; // Iterate over the states in the transition table for (auto const& state_trans : trans_table) { uint32_t num_added = 0; @@ -103,7 +86,6 @@ struct TransducerLookupTable { out_symbol_offsets.push_back(out_symbols.size()); num_added++; } - st++; // Copy the last offset for all symbols (to guarantee a proper lookup for omitted symbols of // this state) @@ -115,30 +97,21 @@ struct TransducerLookupTable { } // Check whether runtime-provided table size exceeds the compile-time given max. table size - if (out_symbols.size() > MAX_TABLE_SIZE) { return cudaErrorInvalidValue; } - - kernel_param.d_trans_offsets = static_cast<OutSymbolOffsetT*>(allocations[MEM_OFFSETS]); - kernel_param.d_out_symbols = static_cast<OutSymbolT*>(allocations[MEM_OUT_SYMBOLS]); - - // Copy out symbols - error = cudaMemcpyAsync(kernel_param.d_trans_offsets, - out_symbol_offsets.data(), - out_symbol_offsets.size() * sizeof(out_symbol_offsets[0]), - cudaMemcpyHostToDevice, - stream); - if (error) { return error; } - - // Copy offsets into output symbols - return cudaMemcpyAsync(kernel_param.d_out_symbols, - out_symbols.data(), - out_symbols.size() * sizeof(out_symbols[0]), - cudaMemcpyHostToDevice, - stream); + if (out_symbols.size() > MAX_TABLE_SIZE) { CUDF_FAIL("Unsupported translation table"); } + + // Prepare host-side data to be copied and passed to the device + std::copy(std::cbegin(out_symbol_offsets), + std::cend(out_symbol_offsets), + translation_table_init.host_ptr()->d_out_offsets); + std::copy(std::cbegin(out_symbols), + std::cend(out_symbols), + translation_table_init.host_ptr()->d_out_symbols); + + // Copy data to device + translation_table_init.host_to_device(stream); } - //------------------------------------------------------------------------------ - // MEMBER VARIABLES - //------------------------------------------------------------------------------ + private: _TempStorage& temp_storage; __device__ __forceinline__ _TempStorage& PrivateStorage() @@ -147,17 +120,19 @@ struct TransducerLookupTable { return private_storage; } - //------------------------------------------------------------------------------ - // CONSTRUCTOR - //------------------------------------------------------------------------------ - __host__ __device__ __forceinline__ TransducerLookupTable(const KernelParameter& kernel_param, - TempStorage& temp_storage) + public: + /** + * @brief Synchronizes the thread block, if called from device, and, hence, requires all threads + * of the thread block to call the constructor + */ + CUDF_HOST_DEVICE TransducerLookupTable(KernelParameter const& kernel_param, + TempStorage& temp_storage) : temp_storage(temp_storage.Alias()) { constexpr uint32_t num_offsets = MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1; #if CUB_PTX_ARCH > 0 for (int i = threadIdx.x; i < num_offsets; i += blockDim.x) { - this->temp_storage.out_offset[i] = kernel_param.d_trans_offsets[i]; + this->temp_storage.out_offset[i] = kernel_param.d_out_offsets[i]; } // Make sure all threads in the block can read out_symbol_offsets[num_offsets - 1] from shared // memory @@ -168,7 +143,7 @@ struct TransducerLookupTable { __syncthreads(); #else for (int i = 0; i < num_offsets; i++) { - this->temp_storage.out_symbol_offsets[i] = kernel_param.d_trans_offsets[i]; + this->temp_storage.out_symbol_offsets[i] = kernel_param.d_out_offsets[i]; } for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; @@ -177,17 +152,17 @@ struct TransducerLookupTable { } template <typename StateIndexT, typename SymbolIndexT, typename RelativeOffsetT> - __host__ __device__ __forceinline__ OutSymbolT operator()(StateIndexT state_id, - SymbolIndexT match_id, - RelativeOffsetT relative_offset) const + constexpr CUDF_HOST_DEVICE OutSymbolT operator()(StateIndexT const state_id, + SymbolIndexT const match_id, + RelativeOffsetT const relative_offset) const { auto offset = temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id] + relative_offset; return temp_storage.out_symbols[offset]; } template <typename StateIndexT, typename SymbolIndexT> - __host__ __device__ __forceinline__ OutSymbolOffsetT operator()(StateIndexT state_id, - SymbolIndexT match_id) const + constexpr CUDF_HOST_DEVICE OutSymbolOffsetT operator()(StateIndexT const state_id, + SymbolIndexT const match_id) const { return temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id + 1] - temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id]; From a064bdd18ecbb04b478ec8379f8458e5f1e4b967 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Mon, 25 Apr 2022 12:17:08 -0700 Subject: [PATCH 46/81] put lookup tables into their own cudf file --- cpp/src/io/fst/symbol_lut.cuh | 158 ------------------------ cpp/src/io/fst/transition_table.cuh | 110 ----------------- cpp/src/io/fst/translation_table.cuh | 175 --------------------------- 3 files changed, 443 deletions(-) delete mode 100644 cpp/src/io/fst/symbol_lut.cuh delete mode 100644 cpp/src/io/fst/transition_table.cuh delete mode 100644 cpp/src/io/fst/translation_table.cuh diff --git a/cpp/src/io/fst/symbol_lut.cuh b/cpp/src/io/fst/symbol_lut.cuh deleted file mode 100644 index abf71a7fbea..00000000000 --- a/cpp/src/io/fst/symbol_lut.cuh +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed 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 <cudf/types.hpp> -#include <io/utilities/hostdevice_vector.hpp> - -#include <cub/cub.cuh> - -#include <algorithm> -#include <cstdint> -#include <vector> - -namespace cudf { -namespace io { -namespace fst { -namespace detail { -/** - * @brief Class template that can be plugged into the finite-state machine to look up the symbol - * group index for a given symbol. Class template does not support multi-symbol lookups (i.e., no - * look-ahead). - * - * @tparam SymbolT The symbol type being passed in to lookup the corresponding symbol group id - */ -template <typename SymbolT> -class SingleSymbolSmemLUT { - private: - // Type used for representing a symbol group id (i.e., what we return for a given symbol) - using SymbolGroupIdT = uint8_t; - - /// Number of entries for every lookup (e.g., for 8-bit Symbol this is 256) - static constexpr uint32_t NUM_ENTRIES_PER_LUT = 0x01U << (sizeof(SymbolT) * 8U); - - struct _TempStorage { - // sym_to_sgid[symbol] -> symbol group index - SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; - }; - - public: - struct KernelParameter { - // sym_to_sgid[min(symbol,num_valid_entries)] -> symbol group index - SymbolT num_valid_entries; - - // sym_to_sgid[symbol] -> symbol group index - SymbolGroupIdT sym_to_sgid[NUM_ENTRIES_PER_LUT]; - }; - - using TempStorage = cub::Uninitialized<_TempStorage>; - - //------------------------------------------------------------------------------ - // HELPER METHODS - //------------------------------------------------------------------------------ - /** - * @brief - * - * @param[out] sgid_init A hostdevice_vector that will be populated - * @param[in] symbol_strings Array of strings, where the i-th string holds all symbols - * (characters!) that correspond to the i-th symbol group index - * @param[in] stream The stream that shall be used to cudaMemcpyAsync the lookup table - * @return - */ - template <typename SymbolGroupItT> - static void InitDeviceSymbolGroupIdLut(hostdevice_vector<KernelParameter>& sgid_init, - SymbolGroupItT const& symbol_strings, - rmm::cuda_stream_view stream) - { - // The symbol group index to be returned if none of the given symbols match - SymbolGroupIdT no_match_id = symbol_strings.size(); - - // The symbol with the largest value that is mapped to a symbol group id - SymbolGroupIdT max_base_match_val = 0; - - // Initialize all entries: by default we return the no-match-id - std::fill(&sgid_init.host_ptr()->sym_to_sgid[0], - &sgid_init.host_ptr()->sym_to_sgid[NUM_ENTRIES_PER_LUT], - no_match_id); - - // Set up lookup table - uint32_t sg_id = 0; - // Iterate over the symbol groups - for (auto const& sg_symbols : symbol_strings) { - // Iterate over all symbols that belong to the current symbol group - for (auto const& sg_symbol : sg_symbols) { - max_base_match_val = std::max(max_base_match_val, static_cast<SymbolGroupIdT>(sg_symbol)); - sgid_init.host_ptr()->sym_to_sgid[static_cast<int32_t>(sg_symbol)] = sg_id; - } - sg_id++; - } - - // Initialize the out-of-bounds lookup: sym_to_sgid[max_base_match_val+1] -> no_match_id - sgid_init.host_ptr()->sym_to_sgid[max_base_match_val + 1] = no_match_id; - - // Alias memory / return memory requiremenets - // TODO I think this could be +1? - sgid_init.host_ptr()->num_valid_entries = max_base_match_val + 2; - - sgid_init.host_to_device(stream); - } - - //------------------------------------------------------------------------------ - // MEMBER VARIABLES - //------------------------------------------------------------------------------ - _TempStorage& temp_storage; - SymbolGroupIdT num_valid_entries; - - //------------------------------------------------------------------------------ - // CONSTRUCTOR - //------------------------------------------------------------------------------ - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - return private_storage; - } - - constexpr CUDF_HOST_DEVICE SingleSymbolSmemLUT(KernelParameter const& kernel_param, - TempStorage& temp_storage) - : temp_storage(temp_storage.Alias()), num_valid_entries(kernel_param.num_valid_entries) - { - // GPU-side init -#if CUB_PTX_ARCH > 0 - for (int32_t i = threadIdx.x; i < kernel_param.num_valid_entries; i += blockDim.x) { - this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; - } - __syncthreads(); - -#else - // CPU-side init - for (std::size_t i = 0; i < kernel_param.num_luts; i++) { - this->temp_storage.sym_to_sgid[i] = kernel_param.sym_to_sgid[i]; - } -#endif - } - - constexpr CUDF_HOST_DEVICE int32_t operator()(SymbolT const symbol) const - { - // Look up the symbol group for given symbol - return temp_storage.sym_to_sgid[min(symbol, num_valid_entries - 1)]; - } -}; - -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf diff --git a/cpp/src/io/fst/transition_table.cuh b/cpp/src/io/fst/transition_table.cuh deleted file mode 100644 index 5eccb926974..00000000000 --- a/cpp/src/io/fst/transition_table.cuh +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed 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 <cudf/types.hpp> -#include <cudf/utilities/error.hpp> -#include <io/utilities/hostdevice_vector.hpp> - -#include <cub/cub.cuh> - -#include <cstdint> - -namespace cudf { -namespace io { -namespace fst { -namespace detail { - -template <int32_t MAX_NUM_SYMBOLS, int32_t MAX_NUM_STATES> -class TransitionTable { - private: - // Type used - using ItemT = char; - - struct _TempStorage { - ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; - }; - - public: - using TempStorage = cub::Uninitialized<_TempStorage>; - - struct KernelParameter { - ItemT transitions[MAX_NUM_STATES * MAX_NUM_SYMBOLS]; - }; - - static void InitDeviceTransitionTable(hostdevice_vector<KernelParameter>& transition_table_init, - const std::vector<std::vector<int>>& trans_table, - rmm::cuda_stream_view stream) - { - // trans_table[state][symbol] -> new state - for (std::size_t state = 0; state < trans_table.size(); ++state) { - for (std::size_t symbol = 0; symbol < trans_table[state].size(); ++symbol) { - transition_table_init.host_ptr()->transitions[symbol * MAX_NUM_STATES + state] = - trans_table[state][symbol]; - } - } - - // Copy transition table to device - transition_table_init.host_to_device(stream); - } - - constexpr CUDF_HOST_DEVICE TransitionTable(const KernelParameter& kernel_param, - TempStorage& temp_storage) - : temp_storage(temp_storage.Alias()) - { -#if CUB_PTX_ARCH > 0 - for (int i = threadIdx.x; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i += blockDim.x) { - this->temp_storage.transitions[i] = kernel_param.transitions[i]; - } - __syncthreads(); -#else - for (int i = 0; i < MAX_NUM_STATES * MAX_NUM_SYMBOLS; i++) { - this->temp_storage.transitions[i] = kernel_param.transitions[i]; - } -#endif - } - - /** - * @brief Returns a random-access iterator to lookup all the state transitions for one specific - * symbol from an arbitrary old_state, i.e., it[old_state] -> new_state. - * - * @param state_id The DFA's current state index from which we'll transition - * @param match_id The symbol group id of the symbol that we just read in - * @return - */ - template <typename StateIndexT, typename SymbolIndexT> - constexpr CUDF_HOST_DEVICE int32_t operator()(StateIndexT const state_id, - SymbolIndexT const match_id) const - { - return temp_storage.transitions[match_id * MAX_NUM_STATES + state_id]; - } - - private: - _TempStorage& temp_storage; - - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - - return private_storage; - } -}; - -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf diff --git a/cpp/src/io/fst/translation_table.cuh b/cpp/src/io/fst/translation_table.cuh deleted file mode 100644 index 89da994606c..00000000000 --- a/cpp/src/io/fst/translation_table.cuh +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed 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 <algorithm> -#include <cudf/types.hpp> -#include <cudf/utilities/error.hpp> -#include <io/utilities/hostdevice_vector.hpp> - -#include "rmm/device_uvector.hpp" - -#include <cub/cub.cuh> - -#include <cstdint> - -namespace cudf { -namespace io { -namespace fst { -namespace detail { - -/** - * @brief Lookup table mapping (old_state, symbol_group_id) transitions to a sequence of symbols - * that the finite-state transducer is supposed to output for each transition - * - * @tparam OutSymbolT The symbol type being output - * @tparam OutSymbolOffsetT Type sufficiently large to index into the lookup table of output symbols - * @tparam MAX_NUM_SYMBOLS The maximum number of symbols being output by a single state transition - * @tparam MAX_NUM_STATES The maximum number of states that this lookup table shall support - * @tparam MAX_TABLE_SIZE The maximum number of items in the lookup table of output symbols - */ -template <typename OutSymbolT, - typename OutSymbolOffsetT, - int32_t MAX_NUM_SYMBOLS, - int32_t MAX_NUM_STATES, - int32_t MAX_TABLE_SIZE = (MAX_NUM_SYMBOLS * MAX_NUM_STATES)> -class TransducerLookupTable { - private: - struct _TempStorage { - OutSymbolOffsetT out_offset[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; - OutSymbolT out_symbols[MAX_TABLE_SIZE]; - }; - - public: - using TempStorage = cub::Uninitialized<_TempStorage>; - - struct KernelParameter { - OutSymbolOffsetT d_out_offsets[MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1]; - OutSymbolT d_out_symbols[MAX_TABLE_SIZE]; - }; - - /** - * @brief Initializes the translation table (both the host and device parts) - */ - static void InitDeviceTranslationTable( - hostdevice_vector<KernelParameter>& translation_table_init, - std::vector<std::vector<std::vector<OutSymbolT>>> const& trans_table, - rmm::cuda_stream_view stream) - { - std::vector<OutSymbolT> out_symbols; - out_symbols.reserve(MAX_TABLE_SIZE); - std::vector<OutSymbolOffsetT> out_symbol_offsets; - out_symbol_offsets.reserve(MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1); - out_symbol_offsets.push_back(0); - - // Iterate over the states in the transition table - for (auto const& state_trans : trans_table) { - uint32_t num_added = 0; - // Iterate over the symbols in the transition table - for (auto const& symbol_out : state_trans) { - // Insert the output symbols for this specific (state, symbol) transition - out_symbols.insert(std::end(out_symbols), std::begin(symbol_out), std::end(symbol_out)); - out_symbol_offsets.push_back(out_symbols.size()); - num_added++; - } - - // Copy the last offset for all symbols (to guarantee a proper lookup for omitted symbols of - // this state) - if (MAX_NUM_SYMBOLS > num_added) { - int32_t count = MAX_NUM_SYMBOLS - num_added; - auto begin_it = std::prev(std::end(out_symbol_offsets)); - std::copy(begin_it, begin_it + count, std::back_inserter(out_symbol_offsets)); - } - } - - // Check whether runtime-provided table size exceeds the compile-time given max. table size - if (out_symbols.size() > MAX_TABLE_SIZE) { CUDF_FAIL("Unsupported translation table"); } - - // Prepare host-side data to be copied and passed to the device - std::copy(std::cbegin(out_symbol_offsets), - std::cend(out_symbol_offsets), - translation_table_init.host_ptr()->d_out_offsets); - std::copy(std::cbegin(out_symbols), - std::cend(out_symbols), - translation_table_init.host_ptr()->d_out_symbols); - - // Copy data to device - translation_table_init.host_to_device(stream); - } - - private: - _TempStorage& temp_storage; - - __device__ __forceinline__ _TempStorage& PrivateStorage() - { - __shared__ _TempStorage private_storage; - return private_storage; - } - - public: - /** - * @brief Synchronizes the thread block, if called from device, and, hence, requires all threads - * of the thread block to call the constructor - */ - CUDF_HOST_DEVICE TransducerLookupTable(KernelParameter const& kernel_param, - TempStorage& temp_storage) - : temp_storage(temp_storage.Alias()) - { - constexpr uint32_t num_offsets = MAX_NUM_STATES * MAX_NUM_SYMBOLS + 1; -#if CUB_PTX_ARCH > 0 - for (int i = threadIdx.x; i < num_offsets; i += blockDim.x) { - this->temp_storage.out_offset[i] = kernel_param.d_out_offsets[i]; - } - // Make sure all threads in the block can read out_symbol_offsets[num_offsets - 1] from shared - // memory - __syncthreads(); - for (int i = threadIdx.x; i < this->temp_storage.out_offset[num_offsets - 1]; i += blockDim.x) { - this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; - } - __syncthreads(); -#else - for (int i = 0; i < num_offsets; i++) { - this->temp_storage.out_symbol_offsets[i] = kernel_param.d_out_offsets[i]; - } - for (int i = 0; i < this->temp_storage.out_symbol_offsets[i]; i++) { - this->temp_storage.out_symbols[i] = kernel_param.d_out_symbols[i]; - } -#endif - } - - template <typename StateIndexT, typename SymbolIndexT, typename RelativeOffsetT> - constexpr CUDF_HOST_DEVICE OutSymbolT operator()(StateIndexT const state_id, - SymbolIndexT const match_id, - RelativeOffsetT const relative_offset) const - { - auto offset = temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id] + relative_offset; - return temp_storage.out_symbols[offset]; - } - - template <typename StateIndexT, typename SymbolIndexT> - constexpr CUDF_HOST_DEVICE OutSymbolOffsetT operator()(StateIndexT const state_id, - SymbolIndexT const match_id) const - { - return temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id + 1] - - temp_storage.out_offset[state_id * MAX_NUM_SYMBOLS + match_id]; - } -}; - -} // namespace detail -} // namespace fst -} // namespace io -} // namespace cudf From 2c729c067c26c48bc90bff6171e013a0bdb99b38 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 2 Jun 2022 05:19:53 -0700 Subject: [PATCH 47/81] fixes sg-count & uses rmm stream in fst tests --- cpp/tests/io/fst/fst_test.cu | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index e198c804222..a668b764793 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -26,6 +26,7 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> +#include <rmm/cuda_stream.hpp> #include <rmm/device_buffer.hpp> #include <rmm/device_uvector.hpp> @@ -211,6 +212,8 @@ TEST_F(FstTest, GroundTruth) auto& d_input = static_cast<cudf::scalar_type_t<std::string>&>(*d_input_string); input = d_input.to_string(stream); + + // Prepare input & output buffers constexpr std::size_t single_item = 1; hostdevice_vector<SymbolT> output_gpu(input.size(), stream_view); From dbefb6c4ce3158cb190026002e45eba548f24e8c Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Tue, 3 May 2022 07:05:44 -0700 Subject: [PATCH 48/81] rebase on latest FST --- cpp/CMakeLists.txt | 1 + cpp/src/io/json/nested_json.h | 116 ++++++++ cpp/src/io/json/nested_json_gpu.cu | 410 +++++++++++++++++++++++++++++ cpp/tests/CMakeLists.txt | 1 + cpp/tests/io/nested_json_test.cu | 189 +++++++++++++ 5 files changed, 717 insertions(+) create mode 100644 cpp/src/io/json/nested_json.h create mode 100644 cpp/src/io/json/nested_json_gpu.cu create mode 100644 cpp/tests/io/nested_json_test.cu diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 0903609c1e2..5d7d63dca78 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -322,6 +322,7 @@ add_library( src/io/csv/writer_impl.cu src/io/functions.cpp src/io/json/json_gpu.cu + src/io/json/nested_json_gpu.cu src/io/json/reader_impl.cu src/io/orc/aggregate_orc_metadata.cpp src/io/orc/dict_enc.cu diff --git a/cpp/src/io/json/nested_json.h b/cpp/src/io/json/nested_json.h new file mode 100644 index 00000000000..58f30c7b9ac --- /dev/null +++ b/cpp/src/io/json/nested_json.h @@ -0,0 +1,116 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <cudf/utilities/span.hpp> + +#include <rmm/cuda_stream_view.hpp> + +namespace cudf { +namespace io { +namespace json { +namespace gpu { + +/// Type used to represent the atomic symbol type used within the finite-state machine +using SymbolT = char; + +/// Type used to represent the stack alphabet (i.e.: empty-stack, struct, list) +using StackSymbolT = char; + +/// Type used to index into the symbols within the JSON input +using SymbolOffsetT = uint32_t; + +/// Type large enough to support indexing up to max nesting level (must be signed) +using StackLevelT = int8_t; + +/// Type used to represent a symbol group id of the input alphabet in the pushdown automaton +using PdaInputSymbolGroupIdT = char; + +/// Type used to represent a symbol group id of the stack alphabet in the pushdown automaton +using PdaStackSymbolGroupIdT = char; + +/// Type used to represent a (input-symbol, stack-symbole)-tuple in stack-symbole-major order +using PdaSymbolGroupIdT = char; + +/// Type being emitted by the pushdown automaton transducer +using PdaTokenT = char; + +/** + * @brief Tokens emitted while parsing a JSON input + */ +enum token_t : PdaTokenT { + /// Beginning-of-struct token (on encounter of semantic '{') + TK_BOS, + /// Beginning-of-list token (on encounter of semantic '[') + TK_BOL, + /// Beginning-of-error token (on first encounter of a parsing error) + TK_ERR, + /// Beginning-of-string-value token (on encounter of the string's first quote) + TK_BST, + /// Beginning-of-value token (first character of literal or numeric) + TK_BOV, + /// End-of-list token (on encounter of semantic ']') + TK_EOL, + /// End-of-struct token (on encounter of semantic '}') + TK_EOS, + /// Beginning-of-field-name token (on encounter of first quote) + TK_BFN, + /// Post-value token (first character after a literal or numeric string) + TK_POV, + /// End-of-string token (on encounter of a string's second quote) + TK_EST, + /// End-of-field-name token (on encounter of a field name's second quote) + TK_EFN, + /// Total number of tokens + NUM_TOKENS +}; + +/** + * @brief Identifies the stack context for each character from a JSON input. Specifically, we + * identify brackets and braces outside of quoted fields (e.g., field names, strings). + * At this stage, we do not perform bracket matching, i.e., we do not verify whether a closing + * bracket would actually pop a the corresponding opening brace. + * + * @param d_json_in The string of input characters + * @param d_top_of_stack + * @param stream The cuda stream to dispatch GPU kernels to + */ +void get_stack_context(device_span<SymbolT const> d_json_in, + device_span<SymbolT> d_top_of_stack, + rmm::cuda_stream_view stream); + +/** + * @brief Parses the given JSON string and emits a sequence of tokens that demarcate relevant + * sections from the input. + * + * @param d_json_in The JSON input + * @param d_tokens_out Device memory to which the parsed tokens are written + * @param d_tokens_indices Device memory to which the indices are written, where each index + * represents the offset within \p d_json_in that cause the input being written + * @param d_num_written_tokens The total number of tokens that were parsed + * @param stream The CUDA stream to which kernels are dispatched + */ +void get_token_stream(device_span<SymbolT const> d_json_in, + device_span<PdaTokenT> d_tokens, + device_span<SymbolOffsetT> d_tokens_indices, + SymbolOffsetT* d_num_written_tokens, + rmm::cuda_stream_view stream); + +} // namespace gpu +} // namespace json +} // namespace io +} // namespace cudf diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu new file mode 100644 index 00000000000..ae1767bf63a --- /dev/null +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -0,0 +1,410 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 "nested_json.h" + +#include <io/fst/logical_stack.cuh> +#include <io/fst/lookup_tables.cuh> +#include <io/utilities/hostdevice_vector.hpp> + +#include <cudf/types.hpp> +#include <cudf/utilities/span.hpp> + +#include <rmm/exec_policy.hpp> + +namespace cudf { +namespace io { +namespace json { +namespace gpu { + +//------------------------------------------------------------------------------ +// JSON-TO-STACK-OP DFA +//------------------------------------------------------------------------------ +namespace to_stack_op { + +/** + * @brief Definition of the DFA's states + */ +enum DFA_STATES { + // The state being active while being outside of a string. When encountering an opening bracket + // or curly brace, we push it onto the stack. When encountering a closing bracket or brace, we + // pop from the stack. + TT_OOS = 0U, + + // The state being active while being within a string (e.g., field name or a string value). We do + // not push or pop from the stack while being in this state. + TT_STR, + + // The state being active after encountering an escape symbol (e.g., '\'), while being in the + // TT_STR state. + TT_ESC, + + // Total number of states + TT_NUM_STATES +}; + +/** + * @brief Definition of the symbol groups + */ +enum DFA_SGID { + OBC = 0U, ///< Opening brace SG: { + OBT, ///< Opening bracket SG: [ + CBC, ///< Closing brace SG: } + CBT, ///< Closing bracket SG: ] + QTE, ///< Quote character SG: " + ESC, ///< Escape character SG: '\' + OTR, ///< SG implicitly matching all other characters + NUM_SYMBOL_GROUPS ///< Total number of symbol groups +}; + +// The i-th string representing all the characters of a symbol group +const std::vector<std::string> symbol_groups = {"{", "[", "}", "]", "\"", "\\"}; + +// Transition table +const std::vector<std::vector<int32_t>> transition_table = { + /* IN_STATE { [ } ] " \ OTHER */ + /* TT_OOS */ {TT_OOS, TT_OOS, TT_OOS, TT_OOS, TT_STR, TT_OOS, TT_OOS}, + /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_STR, TT_STR}, + /* TT_ESC */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR}}; + +// Translation table (i.e., for each transition, what are the symbols that we output) +const std::vector<std::vector<std::vector<char>>> translation_table = { + /* IN_STATE { [ } ] " \ OTHER */ + /* TT_OOS */ {{'{'}, {'['}, {'}'}, {']'}, {'x'}, {'x'}, {'x'}}, + /* TT_STR */ {{'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}}, + /* TT_ESC */ {{'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}, {'x'}}}; + +// The DFA's starting state +constexpr int32_t start_state = TT_OOS; +} // namespace to_stack_op + +//------------------------------------------------------------------------------ +// JSON TOKENIZER PUSHDOWN AUTOMATON +//------------------------------------------------------------------------------ +namespace tokenizer_pda { + +/** + * @brief Symbol groups for the input alphabet for the pushdown automaton + */ +enum SGID : PdaSymbolGroupIdT { + /// Opening brace + OBC, + /// Opening bracket + OBT, + /// Closing brace + CBC, + /// Closing bracket + CBT, + /// Quote + QTE, + /// Escape + ESC, + /// Comma + CMA, + /// Colon + CLN, + /// Whitespace + WSP, + /// Other (any input symbol not assigned to one of the above symbol groups) + OTR, + /// Total number of symbol groups amongst which to differentiate + NUM_PDA_INPUT_SGS +}; + +/** + * @brief Symbols in the stack alphabet + */ +enum STACK_SGID : PdaStackSymbolGroupIdT { + /// Symbol representing the JSON-root (i.e., we're at nesting level '0') + STACK_ROOT = 0, + + /// Symbol representing that we're currently within a list object + STACK_LIST = 1, + + /// Symbol representing that we're currently within a struct object + STACK_STRUCT = 2, + + /// Total number of symbols in the stack alphabet + NUM_STACK_SGS +}; + +/// Total number of symbol groups to differentiate amongst (stack alphabet * input alphabet) +constexpr PdaSymbolGroupIdT NUM_PDA_SGIDS = NUM_PDA_INPUT_SGS * NUM_STACK_SGS; + +/// Mapping a input symbol to the symbol group id +static __constant__ PdaSymbolGroupIdT tos_sg_to_pda_sgid[] = { + OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, WSP, WSP, OTR, OTR, WSP, OTR, OTR, OTR, OTR, OTR, + OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, WSP, OTR, QTE, OTR, OTR, OTR, + OTR, OTR, OTR, OTR, OTR, OTR, CMA, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, + OTR, CLN, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, + OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OBT, ESC, CBT, OTR, + OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, + OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OBC, OTR, CBC, OTR}; + +/** + * @brief Maps a (top-of-stack symbol, input symbol)-pair to a symbol group id of the DVPA + */ +struct PdaSymbolToSymbolGroupId { + template <typename SymbolT, typename StackSymbolT> + __device__ __forceinline__ PdaSymbolGroupIdT + operator()(thrust::tuple<SymbolT, StackSymbolT> symbol_pair) + { + // The symbol read from the input + auto symbol = thrust::get<0>(symbol_pair); + + // The stack symbol (i.e., what is on top of the stack at the time the input symbol was read) + // I.e., whether we're reading in something within a struct, a list, or the JSON root + auto stack_symbol = thrust::get<1>(symbol_pair); + + // The stack symbol offset: '_' is the root group (0), '[' is the list group (1), '{' is the + // struct group (2) + int32_t stack_idx = + (stack_symbol == '_') ? STACK_ROOT : ((stack_symbol == '[') ? STACK_LIST : STACK_STRUCT); + + // The relative symbol group id of the current input symbol + PdaSymbolGroupIdT symbol_gid = tos_sg_to_pda_sgid[min( + static_cast<int32_t>(symbol), + static_cast<int32_t>(sizeof(tos_sg_to_pda_sgid) / sizeof(tos_sg_to_pda_sgid[0])) - 1)]; + return stack_idx * NUM_PDA_INPUT_SGS + symbol_gid; + } +}; + +// The states defined by the pushdown automaton +enum pda_state_t : int32_t { + PD_BOV, + PD_BOA, + PD_LON, + PD_STR, + PD_SCE, + PD_PVL, + PD_BFN, + PD_FLN, + PD_FNE, + PD_PFN, + PD_ERR, + PD_NUM_STATES +}; + +// The starting state of the pushdown automaton +constexpr int32_t start_state = PD_BOV; + +// Identity symbol to symbol group lookup table +const std::vector<std::vector<char>> pda_sgids{ + {0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}, {11}, {12}, {13}, {14}, + {15}, {16}, {17}, {18}, {19}, {20}, {21}, {22}, {23}, {24}, {25}, {26}, {27}, {28}, {29}}; + +/** + * @brief Getting the transition table + */ +std::vector<std::vector<int32_t>> get_transition_table() +{ + std::vector<std::vector<int32_t>> pda_tt(PD_NUM_STATES); + pda_tt[PD_BOV] = {PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON, + PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON, + PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON}; + pda_tt[PD_BOA] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_BOA, PD_BOA, PD_ERR, PD_PVL, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOA, PD_LON, + PD_ERR, PD_ERR, PD_PVL, PD_ERR, PD_FLN, PD_ERR, PD_ERR, PD_ERR, PD_BOA, PD_ERR}; + pda_tt[PD_LON] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_PVL, PD_LON, + PD_ERR, PD_ERR, PD_ERR, PD_PVL, PD_ERR, PD_ERR, PD_BOV, PD_ERR, PD_PVL, PD_LON, + PD_ERR, PD_ERR, PD_PVL, PD_ERR, PD_ERR, PD_ERR, PD_BFN, PD_ERR, PD_PVL, PD_LON}; + pda_tt[PD_STR] = {PD_STR, PD_STR, PD_STR, PD_STR, PD_PVL, PD_SCE, PD_STR, PD_STR, PD_STR, PD_STR, + PD_STR, PD_STR, PD_STR, PD_STR, PD_PVL, PD_SCE, PD_STR, PD_STR, PD_STR, PD_STR, + PD_STR, PD_STR, PD_STR, PD_STR, PD_PVL, PD_SCE, PD_STR, PD_STR, PD_STR, PD_STR}; + pda_tt[PD_SCE] = {PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, + PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, + PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR, PD_STR}; + pda_tt[PD_PVL] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_PVL, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_PVL, PD_ERR, PD_ERR, PD_BOV, PD_ERR, PD_PVL, PD_ERR, + PD_ERR, PD_ERR, PD_PVL, PD_ERR, PD_ERR, PD_ERR, PD_BFN, PD_ERR, PD_PVL, PD_ERR}; + pda_tt[PD_BFN] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_FLN, PD_ERR, PD_ERR, PD_ERR, PD_BFN, PD_ERR}; + pda_tt[PD_FLN] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_FLN, PD_FLN, PD_FLN, PD_FLN, PD_PFN, PD_FNE, PD_FLN, PD_FLN, PD_FLN, PD_FLN}; + pda_tt[PD_FNE] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_FLN, PD_FLN, PD_FLN, PD_FLN, PD_FLN, PD_FLN, PD_FLN, PD_FLN, PD_FLN, PD_FLN}; + pda_tt[PD_PFN] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_PFN, PD_ERR}; + pda_tt[PD_ERR] = {PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, + PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR, PD_ERR}; + return pda_tt; +} + +/** + * @brief Getting the translation table + */ +std::vector<std::vector<std::vector<char>>> get_translation_table() +{ + std::vector<std::vector<std::vector<char>>> pda_tlt(PD_NUM_STATES); + pda_tlt[PD_BOV] = {{TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_ERR}, {TK_BST}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {}, {TK_BOV}, {TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_ERR}, {TK_BST}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {}, {TK_BOV}, {TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_ERR}, + {TK_BST}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {TK_BOV}}; + pda_tlt[PD_BOA] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_EOL}, {TK_BST}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {}, {TK_BOV}, {TK_ERR}, {TK_ERR}, {TK_EOS}, {TK_ERR}, + {TK_BFN}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {TK_ERR}}; + pda_tlt[PD_LON] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_POV}, {}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_POV, TK_EOL}, {TK_ERR}, + {TK_ERR}, {TK_POV}, {TK_ERR}, {TK_POV}, {}, + {TK_ERR}, {TK_ERR}, {TK_POV, TK_EOS}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_POV}, {TK_ERR}, {TK_POV}, {}}; + pda_tlt[PD_STR] = {{}, {}, {}, {}, {TK_EST}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {TK_EST}, + {}, {}, {}, {}, {}, {}, {}, {}, {}, {TK_EST}, {}, {}, {}, {}, {}}; + pda_tlt[PD_SCE] = {{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, + {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}}; + pda_tlt[PD_PVL] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_EOL}, {TK_ERR}, {TK_ERR}, + {}, {TK_ERR}, {}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_EOS}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {}, {TK_ERR}, {}, {TK_ERR}}; + pda_tlt[PD_BFN] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_BFN}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {TK_ERR}}; + pda_tlt[PD_FLN] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {}, {}, {}, + {TK_EFN}, {}, {}, {}, {}, {}}; + pda_tlt[PD_FNE] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {}, {}, {}, + {}, {}, {}, {}, {}, {}}; + pda_tlt[PD_PFN] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, + {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {}, {TK_ERR}}; + pda_tlt[PD_ERR] = {{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, + {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}}; + return pda_tlt; +} + +} // namespace tokenizer_pda + +/** + * @brief Function object used to filter for brackets and braces that represent push and pop + * operations + * + */ +struct JSONToStackOp { + template <typename StackSymbolT> + constexpr CUDF_HOST_DEVICE fst::stack_op_type operator()(StackSymbolT const& stack_symbol) const + { + return (stack_symbol == '{' || stack_symbol == '[') ? fst::stack_op_type::PUSH + : (stack_symbol == '}' || stack_symbol == ']') ? fst::stack_op_type::POP + : fst::stack_op_type::READ; + } +}; + +void get_stack_context(device_span<SymbolT const> d_json_in, + device_span<SymbolT> d_top_of_stack, + rmm::cuda_stream_view stream) +{ + constexpr std::size_t single_item = 1; + + // Symbol that will represent empty-stack (i.e., that we're at the DOM root) + constexpr StackSymbolT root_symbol = '_'; + // This can be any stack symbol from the stack alphabet that does not push onto stack + constexpr StackSymbolT read_symbol = 'x'; + + // Number of stack operations in the input (i.e., number of '{', '}', '[', ']' outside of quotes) + hostdevice_vector<SymbolOffsetT> d_num_stack_ops(single_item, stream); + + // Sequence of stack symbols and their position in the original input (sparse representation) + rmm::device_uvector<StackSymbolT> d_stack_ops{d_json_in.size(), stream}; + rmm::device_uvector<SymbolOffsetT> d_stack_op_indices{d_json_in.size(), stream}; + + // Prepare finite-state transducer that only selects '{', '}', '[', ']' outside of quotes + using ToStackOpFstT = cudf::io::fst::detail::Dfa<StackSymbolT, + (to_stack_op::DFA_SGID::NUM_SYMBOL_GROUPS - 1), + to_stack_op::DFA_STATES::TT_NUM_STATES>; + ToStackOpFstT json_to_stack_ops_fst{to_stack_op::symbol_groups, + to_stack_op::transition_table, + to_stack_op::translation_table, + stream}; + + // "Search" for relevant occurrence of brackets and braces that indicate the beginning/end + // structs/lists + json_to_stack_ops_fst.Transduce(d_json_in.begin(), + static_cast<SymbolOffsetT>(d_json_in.size()), + d_stack_ops.data(), + d_stack_op_indices.data(), + d_num_stack_ops.device_ptr(), + to_stack_op::start_state, + stream); + + // Request temporary storage requirements + fst::sparse_stack_op_to_top_of_stack<StackLevelT>( + d_stack_ops.data(), + device_span<SymbolOffsetT>{d_stack_op_indices.data(), d_stack_op_indices.size()}, + JSONToStackOp{}, + d_top_of_stack.data(), + root_symbol, + read_symbol, + d_json_in.size(), + stream); +} + +void get_token_stream(device_span<SymbolT const> d_json_in, + device_span<PdaTokenT> d_tokens, + device_span<SymbolOffsetT> d_tokens_indices, + SymbolOffsetT* d_num_written_tokens, + rmm::cuda_stream_view stream) +{ + // Memory holding the top-of-stack stack context for the input + rmm::device_uvector<StackSymbolT> d_top_of_stack{d_json_in.size(), stream}; + + // Identify what is the stack context for each input character (is it: JSON-root, struct, or list) + get_stack_context(d_json_in, d_top_of_stack, stream); + + // Prepare for PDA transducer pass, merging input symbols with stack symbols + rmm::device_uvector<PdaSymbolGroupIdT> d_pda_sgids{d_json_in.size(), stream}; + auto zip_in = thrust::make_zip_iterator(d_json_in.data(), d_top_of_stack.data()); + thrust::transform(rmm::exec_policy(stream), + zip_in, + zip_in + d_json_in.size(), + d_pda_sgids.data(), + tokenizer_pda::PdaSymbolToSymbolGroupId{}); + + // PDA transducer alias + using ToTokenStreamFstT = cudf::io::fst::detail:: + Dfa<StackSymbolT, (tokenizer_pda::NUM_PDA_SGIDS - 1), tokenizer_pda::PD_NUM_STATES>; + + // Instantiating PDA transducer + ToTokenStreamFstT json_to_tokens_fst{tokenizer_pda::pda_sgids, + tokenizer_pda::get_transition_table(), + tokenizer_pda::get_translation_table(), + stream}; + + // Perform a PDA-transducer pass + json_to_tokens_fst.Transduce(d_pda_sgids.begin(), + static_cast<SymbolOffsetT>(d_json_in.size()), + d_tokens.data(), + d_tokens_indices.data(), + d_num_written_tokens, + tokenizer_pda::start_state, + stream); +} + +} // namespace gpu +} // namespace json +} // namespace io +} // namespace cudf diff --git a/cpp/tests/CMakeLists.txt b/cpp/tests/CMakeLists.txt index 587f317692f..6c3d2353693 100644 --- a/cpp/tests/CMakeLists.txt +++ b/cpp/tests/CMakeLists.txt @@ -222,6 +222,7 @@ ConfigureTest(FILE_IO_TEST io/file_io_test.cpp) ConfigureTest(ORC_TEST io/orc_test.cpp) ConfigureTest(PARQUET_TEST io/parquet_test.cpp) ConfigureTest(JSON_TEST io/json_test.cpp) +ConfigureTest(NESTED_JSON_TEST io/nested_json_test.cu) ConfigureTest(ARROW_IO_SOURCE_TEST io/arrow_io_source_test.cpp) ConfigureTest(MULTIBYTE_SPLIT_TEST io/text/multibyte_split_test.cpp) ConfigureTest(LOGICAL_STACK_TEST io/fst/logical_stack_test.cu) diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu new file mode 100644 index 00000000000..6336f493c17 --- /dev/null +++ b/cpp/tests/io/nested_json_test.cu @@ -0,0 +1,189 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <io/json/nested_json.h> +#include <io/utilities/hostdevice_vector.hpp> + +#include <cudf_test/base_fixture.hpp> +#include <cudf_test/cudf_gtest.hpp> + +namespace nested_json = cudf::io::json::gpu; + +// Base test fixture for tests +struct JsonTest : public cudf::test::BaseFixture { +}; + +TEST_F(JsonTest, StackContext) +{ + // Type used to represent the atomic symbol type used within the finite-state machine + using SymbolT = char; + using StackSymbolT = char; + + // Prepare cuda stream for data transfers & kernels + cudaStream_t stream = nullptr; + cudaStreamCreate(&stream); + rmm::cuda_stream_view stream_view(stream); + + // Test input + std::string input = R"( [{)" + R"("category": "reference",)" + R"("index:": [4,12,42],)" + R"("author": "Nigel Rees",)" + R"("title": "[Sayings of the Century]",)" + R"("price": 8.95)" + R"(}, )" + R"({)" + R"("category": "reference",)" + R"("index": [4,{},null,{"a":[{ }, {}] } ],)" + R"("author": "Nigel Rees",)" + R"("title": "{}[], <=semantic-symbols-string",)" + R"("price": 8.95)" + R"(}] )"; + + // Prepare input & output buffers + rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); + hostdevice_vector<StackSymbolT> stack_context(input.size(), stream_view); + + ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( + d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); + + // Run algorithm + cudf::io::json::gpu::get_stack_context( + d_input, + cudf::device_span<StackSymbolT>{stack_context.device_ptr(), stack_context.size()}, + stream); + + // Copy back the results + stack_context.device_to_host(stream); + + // Make sure we copied back the stack context + stream_view.synchronize(); + + std::vector<char> golden_stack_context{ + '_', '_', '_', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '[', '[', '[', '[', '[', '[', '[', '[', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', '[', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', + '{', '[', '[', '[', '[', '[', '[', '[', '{', '{', '{', '{', '{', '[', '{', '{', '[', '[', + '[', '{', '[', '{', '{', '[', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '_'}; + + ASSERT_EQ(golden_stack_context.size(), stack_context.size()); + for (std::size_t i = 0; i < stack_context.size() && i < 1000; i++) { + ASSERT_EQ(golden_stack_context[i], stack_context[i]); + } +} + +TEST_F(JsonTest, TokenStream) +{ + using cudf::io::json::gpu::PdaTokenT; + using cudf::io::json::gpu::SymbolOffsetT; + using cudf::io::json::gpu::SymbolT; + + constexpr std::size_t single_item = 1; + + // Prepare cuda stream for data transfers & kernels + cudaStream_t stream = nullptr; + cudaStreamCreate(&stream); + rmm::cuda_stream_view stream_view(stream); + + // Test input + std::string input = R"( [{)" + R"("category": "reference",)" + R"("index:": [4,12,42],)" + R"("author": "Nigel Rees",)" + R"("title": "[Sayings of the Century]",)" + R"("price": 8.95)" + R"(}, )" + R"({)" + R"("category": "reference",)" + R"("index": [4,{},null,{"a":[{ }, {}] } ],)" + R"("author": "Nigel Rees",)" + R"("title": "{}[], <=semantic-symbols-string",)" + R"("price": 8.95)" + R"(}] )"; + + // Prepare input & output buffers + rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); + + ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( + d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); + + + hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream}; + hostdevice_vector<SymbolOffsetT> token_indices_gpu{input.size(), stream}; + hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream}; + + // Parse the JSON and get the token stream + cudf::io::json::gpu::get_token_stream( + d_input, + cudf::device_span<PdaTokenT>{tokens_gpu.device_ptr(), tokens_gpu.size()}, + cudf::device_span<SymbolOffsetT>{token_indices_gpu.device_ptr(), token_indices_gpu.size()}, + num_tokens_out.device_ptr(), + stream); + + // Copy back the number of tokens that were written + num_tokens_out.device_to_host(stream); + tokens_gpu.device_to_host(stream); + token_indices_gpu.device_to_host(stream); + + // Make sure we copied back all relevant data + stream_view.synchronize(); + + // Golden token stream sample + std::vector<std::pair<std::size_t, nested_json::PdaTokenT>> golden_token_stream = { + {2, nested_json::TK_BOL}, {3, nested_json::TK_BOS}, {4, nested_json::TK_BFN}, + {13, nested_json::TK_EFN}, {16, nested_json::TK_BST}, {26, nested_json::TK_EST}, + {28, nested_json::TK_BFN}, {35, nested_json::TK_EFN}, {38, nested_json::TK_BOL}, + {39, nested_json::TK_BOV}, {40, nested_json::TK_POV}, {41, nested_json::TK_BOV}, + {43, nested_json::TK_POV}, {44, nested_json::TK_BOV}, {46, nested_json::TK_POV}, + {46, nested_json::TK_EOL}, {48, nested_json::TK_BFN}, {55, nested_json::TK_EFN}, + {58, nested_json::TK_BST}, {69, nested_json::TK_EST}, {71, nested_json::TK_BFN}, + {77, nested_json::TK_EFN}, {80, nested_json::TK_BST}, {105, nested_json::TK_EST}, + {107, nested_json::TK_BFN}, {113, nested_json::TK_EFN}, {116, nested_json::TK_BOV}, + {120, nested_json::TK_POV}, {120, nested_json::TK_EOS}, {124, nested_json::TK_BOS}, + {125, nested_json::TK_BFN}, {134, nested_json::TK_EFN}, {137, nested_json::TK_BST}, + {147, nested_json::TK_EST}, {149, nested_json::TK_BFN}, {155, nested_json::TK_EFN}, + {158, nested_json::TK_BOL}, {159, nested_json::TK_BOV}, {160, nested_json::TK_POV}, + {161, nested_json::TK_BOS}, {162, nested_json::TK_EOS}, {164, nested_json::TK_BOV}, + {168, nested_json::TK_POV}, {169, nested_json::TK_BOS}, {170, nested_json::TK_BFN}, + {172, nested_json::TK_EFN}, {174, nested_json::TK_BOL}, {175, nested_json::TK_BOS}, + {177, nested_json::TK_EOS}, {180, nested_json::TK_BOS}, {181, nested_json::TK_EOS}, + {182, nested_json::TK_EOL}, {184, nested_json::TK_EOS}, {186, nested_json::TK_EOL}, + {188, nested_json::TK_BFN}, {195, nested_json::TK_EFN}, {198, nested_json::TK_BST}, + {209, nested_json::TK_EST}, {211, nested_json::TK_BFN}, {217, nested_json::TK_EFN}, + {220, nested_json::TK_BST}, {252, nested_json::TK_EST}, {254, nested_json::TK_BFN}, + {260, nested_json::TK_EFN}, {263, nested_json::TK_BOV}, {267, nested_json::TK_POV}, + {267, nested_json::TK_EOS}, {268, nested_json::TK_EOL}}; + + // Verify the number of tokens matches + ASSERT_EQ(golden_token_stream.size(), num_tokens_out[0]); + + for (std::size_t i = 0; i < num_tokens_out[0]; i++) { + // Ensure the index the tokens are pointing to do match + ASSERT_EQ(golden_token_stream[i].first, token_indices_gpu[i]); + // Ensure the token category is correct + ASSERT_EQ(golden_token_stream[i].second, tokens_gpu[i]); + } +} From d54f3e5a21431a1dd27c1090ef905051bf4636a7 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 2 Jun 2022 08:19:37 -0700 Subject: [PATCH 49/81] fixes breaking changes from dependent-FST-PR --- cpp/src/io/json/nested_json_gpu.cu | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index ae1767bf63a..181914ae8d9 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -334,7 +334,7 @@ void get_stack_context(device_span<SymbolT const> d_json_in, // Prepare finite-state transducer that only selects '{', '}', '[', ']' outside of quotes using ToStackOpFstT = cudf::io::fst::detail::Dfa<StackSymbolT, - (to_stack_op::DFA_SGID::NUM_SYMBOL_GROUPS - 1), + to_stack_op::DFA_SGID::NUM_SYMBOL_GROUPS, to_stack_op::DFA_STATES::TT_NUM_STATES>; ToStackOpFstT json_to_stack_ops_fst{to_stack_op::symbol_groups, to_stack_op::transition_table, @@ -386,7 +386,7 @@ void get_token_stream(device_span<SymbolT const> d_json_in, // PDA transducer alias using ToTokenStreamFstT = cudf::io::fst::detail:: - Dfa<StackSymbolT, (tokenizer_pda::NUM_PDA_SGIDS - 1), tokenizer_pda::PD_NUM_STATES>; + Dfa<StackSymbolT, tokenizer_pda::NUM_PDA_SGIDS, tokenizer_pda::PD_NUM_STATES>; // Instantiating PDA transducer ToTokenStreamFstT json_to_tokens_fst{tokenizer_pda::pda_sgids, From 5fc339953bc7d76781163d27c7edaa24291f5c87 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 13 Jul 2022 05:45:49 -0700 Subject: [PATCH 50/81] fixes for breaking downstream interface changes --- cpp/src/io/json/nested_json_gpu.cu | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 181914ae8d9..88d0aafb793 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -35,10 +35,13 @@ namespace gpu { //------------------------------------------------------------------------------ namespace to_stack_op { +// Type used to represent the target state in the transition table +using StateT = char; + /** * @brief Definition of the DFA's states */ -enum DFA_STATES { +enum DFA_STATES : StateT { // The state being active while being outside of a string. When encountering an opening bracket // or curly brace, we push it onto the stack. When encountering a closing bracket or brace, we // pop from the stack. @@ -74,7 +77,7 @@ enum DFA_SGID { const std::vector<std::string> symbol_groups = {"{", "[", "}", "]", "\"", "\\"}; // Transition table -const std::vector<std::vector<int32_t>> transition_table = { +const std::vector<std::vector<StateT>> transition_table = { /* IN_STATE { [ } ] " \ OTHER */ /* TT_OOS */ {TT_OOS, TT_OOS, TT_OOS, TT_OOS, TT_STR, TT_OOS, TT_OOS}, /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_STR, TT_STR}, @@ -96,6 +99,9 @@ constexpr int32_t start_state = TT_OOS; //------------------------------------------------------------------------------ namespace tokenizer_pda { +// Type used to represent the target state in the transition table +using StateT = char; + /** * @brief Symbol groups for the input alphabet for the pushdown automaton */ @@ -183,7 +189,7 @@ struct PdaSymbolToSymbolGroupId { }; // The states defined by the pushdown automaton -enum pda_state_t : int32_t { +enum pda_state_t : StateT { PD_BOV, PD_BOA, PD_LON, @@ -199,7 +205,7 @@ enum pda_state_t : int32_t { }; // The starting state of the pushdown automaton -constexpr int32_t start_state = PD_BOV; +constexpr StateT start_state = PD_BOV; // Identity symbol to symbol group lookup table const std::vector<std::vector<char>> pda_sgids{ @@ -207,11 +213,11 @@ const std::vector<std::vector<char>> pda_sgids{ {15}, {16}, {17}, {18}, {19}, {20}, {21}, {22}, {23}, {24}, {25}, {26}, {27}, {28}, {29}}; /** - * @brief Getting the transition table + * @brief Getting the transition table */ -std::vector<std::vector<int32_t>> get_transition_table() +std::vector<std::vector<StateT>> get_transition_table() { - std::vector<std::vector<int32_t>> pda_tt(PD_NUM_STATES); + std::vector<std::vector<StateT>> pda_tt(PD_NUM_STATES); pda_tt[PD_BOV] = {PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON, PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON, PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON}; @@ -249,7 +255,7 @@ std::vector<std::vector<int32_t>> get_transition_table() } /** - * @brief Getting the translation table + * @brief Getting the translation table */ std::vector<std::vector<std::vector<char>>> get_translation_table() { From 6f659477b0975a48b29670e01d92ca7f768e42f5 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 13 Jul 2022 07:22:52 -0700 Subject: [PATCH 51/81] wraps if with stream params into detail ns --- cpp/src/io/json/nested_json.h | 12 ++++-------- cpp/src/io/json/nested_json_gpu.cu | 14 ++++++-------- cpp/tests/io/nested_json_test.cu | 7 +++---- 3 files changed, 13 insertions(+), 20 deletions(-) diff --git a/cpp/src/io/json/nested_json.h b/cpp/src/io/json/nested_json.h index 58f30c7b9ac..b623ad3ea41 100644 --- a/cpp/src/io/json/nested_json.h +++ b/cpp/src/io/json/nested_json.h @@ -20,10 +20,7 @@ #include <rmm/cuda_stream_view.hpp> -namespace cudf { -namespace io { -namespace json { -namespace gpu { +namespace cudf::io::json::gpu { /// Type used to represent the atomic symbol type used within the finite-state machine using SymbolT = char; @@ -79,6 +76,7 @@ enum token_t : PdaTokenT { NUM_TOKENS }; +namespace detail { /** * @brief Identifies the stack context for each character from a JSON input. Specifically, we * identify brackets and braces outside of quoted fields (e.g., field names, strings). @@ -109,8 +107,6 @@ void get_token_stream(device_span<SymbolT const> d_json_in, device_span<SymbolOffsetT> d_tokens_indices, SymbolOffsetT* d_num_written_tokens, rmm::cuda_stream_view stream); +} // namespace detail -} // namespace gpu -} // namespace json -} // namespace io -} // namespace cudf +} // namespace cudf::io::json::gpu diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 88d0aafb793..d0888ca1bc1 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -25,10 +25,7 @@ #include <rmm/exec_policy.hpp> -namespace cudf { -namespace io { -namespace json { -namespace gpu { +namespace cudf::io::json::gpu { //------------------------------------------------------------------------------ // JSON-TO-STACK-OP DFA @@ -320,6 +317,8 @@ struct JSONToStackOp { } }; +namespace detail { + void get_stack_context(device_span<SymbolT const> d_json_in, device_span<SymbolT> d_top_of_stack, rmm::cuda_stream_view stream) @@ -410,7 +409,6 @@ void get_token_stream(device_span<SymbolT const> d_json_in, stream); } -} // namespace gpu -} // namespace json -} // namespace io -} // namespace cudf +} // namespace detail + +} // namespace cudf::io::json::gpu diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 6336f493c17..5a0e39d32b5 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -61,7 +61,7 @@ TEST_F(JsonTest, StackContext) d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); // Run algorithm - cudf::io::json::gpu::get_stack_context( + cudf::io::json::gpu::detail::get_stack_context( d_input, cudf::device_span<StackSymbolT>{stack_context.device_ptr(), stack_context.size()}, stream); @@ -130,13 +130,12 @@ TEST_F(JsonTest, TokenStream) ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); - hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream}; hostdevice_vector<SymbolOffsetT> token_indices_gpu{input.size(), stream}; hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream}; // Parse the JSON and get the token stream - cudf::io::json::gpu::get_token_stream( + cudf::io::json::gpu::detail::get_token_stream( d_input, cudf::device_span<PdaTokenT>{tokens_gpu.device_ptr(), tokens_gpu.size()}, cudf::device_span<SymbolOffsetT>{token_indices_gpu.device_ptr(), token_indices_gpu.size()}, @@ -179,7 +178,7 @@ TEST_F(JsonTest, TokenStream) // Verify the number of tokens matches ASSERT_EQ(golden_token_stream.size(), num_tokens_out[0]); - + for (std::size_t i = 0; i < num_tokens_out[0]; i++) { // Ensure the index the tokens are pointing to do match ASSERT_EQ(golden_token_stream[i].first, token_indices_gpu[i]); From 6ffc7f3d6ed98b9c36633a55c77de4b30f0f2137 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 14 Jul 2022 04:15:11 -0700 Subject: [PATCH 52/81] renames enums & moving from device_span to ptr params --- cpp/src/io/json/nested_json.h | 44 ++-- cpp/src/io/json/nested_json_gpu.cu | 317 ++++++++++++++++++++++++----- cpp/tests/io/nested_json_test.cu | 63 +++--- 3 files changed, 315 insertions(+), 109 deletions(-) diff --git a/cpp/src/io/json/nested_json.h b/cpp/src/io/json/nested_json.h index b623ad3ea41..6bc99d50bd8 100644 --- a/cpp/src/io/json/nested_json.h +++ b/cpp/src/io/json/nested_json.h @@ -51,27 +51,27 @@ using PdaTokenT = char; */ enum token_t : PdaTokenT { /// Beginning-of-struct token (on encounter of semantic '{') - TK_BOS, + StructBegin, /// Beginning-of-list token (on encounter of semantic '[') - TK_BOL, + ListBegin, /// Beginning-of-error token (on first encounter of a parsing error) - TK_ERR, + ErrorBegin, /// Beginning-of-string-value token (on encounter of the string's first quote) - TK_BST, + StringBegin, /// Beginning-of-value token (first character of literal or numeric) - TK_BOV, + ValueBegin, /// End-of-list token (on encounter of semantic ']') - TK_EOL, + ListEnd, /// End-of-struct token (on encounter of semantic '}') - TK_EOS, + StructEnd, /// Beginning-of-field-name token (on encounter of first quote) - TK_BFN, + FieldNameBegin, /// Post-value token (first character after a literal or numeric string) - TK_POV, + ValueEnd, /// End-of-string token (on encounter of a string's second quote) - TK_EST, + StringEnd, /// End-of-field-name token (on encounter of a field name's second quote) - TK_EFN, + FieldNameEnd, /// Total number of tokens NUM_TOKENS }; @@ -83,28 +83,28 @@ namespace detail { * At this stage, we do not perform bracket matching, i.e., we do not verify whether a closing * bracket would actually pop a the corresponding opening brace. * - * @param d_json_in The string of input characters - * @param d_top_of_stack - * @param stream The cuda stream to dispatch GPU kernels to + * @param[in] d_json_in The string of input characters + * @param[out] d_top_of_stack + * @param[in] stream The cuda stream to dispatch GPU kernels to */ void get_stack_context(device_span<SymbolT const> d_json_in, - device_span<SymbolT> d_top_of_stack, + SymbolT* d_top_of_stack, rmm::cuda_stream_view stream); /** * @brief Parses the given JSON string and emits a sequence of tokens that demarcate relevant * sections from the input. * - * @param d_json_in The JSON input - * @param d_tokens_out Device memory to which the parsed tokens are written - * @param d_tokens_indices Device memory to which the indices are written, where each index + * @param[in] d_json_in The JSON input + * @param[out] d_tokens_out Device memory to which the parsed tokens are written + * @param[out] d_tokens_indices Device memory to which the indices are written, where each index * represents the offset within \p d_json_in that cause the input being written - * @param d_num_written_tokens The total number of tokens that were parsed - * @param stream The CUDA stream to which kernels are dispatched + * @param[out] d_num_written_tokens The total number of tokens that were parsed + * @param[in] stream The CUDA stream to which kernels are dispatched */ void get_token_stream(device_span<SymbolT const> d_json_in, - device_span<PdaTokenT> d_tokens, - device_span<SymbolOffsetT> d_tokens_indices, + PdaTokenT* d_tokens, + SymbolOffsetT* d_tokens_indices, SymbolOffsetT* d_num_written_tokens, rmm::cuda_stream_view stream); } // namespace detail diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index d0888ca1bc1..07994ea2fa7 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -59,14 +59,14 @@ enum DFA_STATES : StateT { /** * @brief Definition of the symbol groups */ -enum DFA_SGID { - OBC = 0U, ///< Opening brace SG: { - OBT, ///< Opening bracket SG: [ - CBC, ///< Closing brace SG: } - CBT, ///< Closing bracket SG: ] - QTE, ///< Quote character SG: " - ESC, ///< Escape character SG: '\' - OTR, ///< SG implicitly matching all other characters +enum class DFASymbolGroupID : uint32_t { + OpenBrace, ///< Opening brace SG: { + OpenBracket, ///< Opening bracket SG: [ + CloseBrace, ///< Closing brace SG: } + CloseBracket, ///< Closing bracket SG: ] + Quote, ///< Quote character SG: " + Escape, ///< Escape character SG: '\' + Other, ///< SG implicitly matching all other characters NUM_SYMBOL_GROUPS ///< Total number of symbol groups }; @@ -257,44 +257,251 @@ std::vector<std::vector<StateT>> get_transition_table() std::vector<std::vector<std::vector<char>>> get_translation_table() { std::vector<std::vector<std::vector<char>>> pda_tlt(PD_NUM_STATES); - pda_tlt[PD_BOV] = {{TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_ERR}, {TK_BST}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {}, {TK_BOV}, {TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_ERR}, {TK_BST}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {}, {TK_BOV}, {TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_ERR}, - {TK_BST}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {TK_BOV}}; - pda_tlt[PD_BOA] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_BOS}, {TK_BOL}, {TK_ERR}, {TK_EOL}, {TK_BST}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {}, {TK_BOV}, {TK_ERR}, {TK_ERR}, {TK_EOS}, {TK_ERR}, - {TK_BFN}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {TK_ERR}}; - pda_tlt[PD_LON] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_POV}, {}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_POV, TK_EOL}, {TK_ERR}, - {TK_ERR}, {TK_POV}, {TK_ERR}, {TK_POV}, {}, - {TK_ERR}, {TK_ERR}, {TK_POV, TK_EOS}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_POV}, {TK_ERR}, {TK_POV}, {}}; - pda_tlt[PD_STR] = {{}, {}, {}, {}, {TK_EST}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {TK_EST}, - {}, {}, {}, {}, {}, {}, {}, {}, {}, {TK_EST}, {}, {}, {}, {}, {}}; + pda_tlt[PD_BOV] = {{token_t::StructBegin}, + {token_t::ListBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::StringBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ValueBegin}, + {token_t::StructBegin}, + {token_t::ListBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::StringBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ValueBegin}, + {token_t::StructBegin}, + {token_t::ListBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::StringBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ValueBegin}}; + pda_tlt[PD_BOA] = {{token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::StructBegin}, + {token_t::ListBegin}, + {token_t::ErrorBegin}, + {token_t::ListEnd}, + {token_t::StringBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ValueBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::StructEnd}, + {token_t::ErrorBegin}, + {token_t::FieldNameBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ErrorBegin}}; + pda_tlt[PD_LON] = {{token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ValueEnd}, + {}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ValueEnd, token_t::ListEnd}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ValueEnd}, + {token_t::ErrorBegin}, + {token_t::ValueEnd}, + {}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ValueEnd, token_t::StructEnd}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ValueEnd}, + {token_t::ErrorBegin}, + {token_t::ValueEnd}, + {}}; + pda_tlt[PD_STR] = {{}, {}, {}, {}, {token_t::StringEnd}, {}, {}, {}, {}, {}, + {}, {}, {}, {}, {token_t::StringEnd}, {}, {}, {}, {}, {}, + {}, {}, {}, {}, {token_t::StringEnd}, {}, {}, {}, {}, {}}; pda_tlt[PD_SCE] = {{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}}; - pda_tlt[PD_PVL] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_EOL}, {TK_ERR}, {TK_ERR}, - {}, {TK_ERR}, {}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_EOS}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {}, {TK_ERR}, {}, {TK_ERR}}; - pda_tlt[PD_BFN] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_BFN}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {TK_ERR}}; - pda_tlt[PD_FLN] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {}, {}, {}, - {TK_EFN}, {}, {}, {}, {}, {}}; - pda_tlt[PD_FNE] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {}, {}, {}, - {}, {}, {}, {}, {}, {}}; - pda_tlt[PD_PFN] = {{TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, {TK_ERR}, - {TK_ERR}, {TK_ERR}, {TK_ERR}, {}, {}, {TK_ERR}}; + pda_tlt[PD_PVL] = {{token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ListEnd}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ErrorBegin}, + {}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::StructEnd}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ErrorBegin}, + {}, + {token_t::ErrorBegin}}; + pda_tlt[PD_BFN] = {{token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::FieldNameBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {token_t::ErrorBegin}}; + pda_tlt[PD_FLN] = {{token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {}, + {}, + {}, + {token_t::FieldNameEnd}, + {}, + {}, + {}, + {}, + {}}; + pda_tlt[PD_FNE] = {{token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {}, + {}, + {}, + {}, + {}, + {}, + {}, + {}, + {}}; + pda_tlt[PD_PFN] = {{token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {token_t::ErrorBegin}, + {}, + {}, + {token_t::ErrorBegin}}; pda_tlt[PD_ERR] = {{}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}}; return pda_tlt; @@ -320,7 +527,7 @@ struct JSONToStackOp { namespace detail { void get_stack_context(device_span<SymbolT const> d_json_in, - device_span<SymbolT> d_top_of_stack, + SymbolT* d_top_of_stack, rmm::cuda_stream_view stream) { constexpr std::size_t single_item = 1; @@ -338,9 +545,11 @@ void get_stack_context(device_span<SymbolT const> d_json_in, rmm::device_uvector<SymbolOffsetT> d_stack_op_indices{d_json_in.size(), stream}; // Prepare finite-state transducer that only selects '{', '}', '[', ']' outside of quotes - using ToStackOpFstT = cudf::io::fst::detail::Dfa<StackSymbolT, - to_stack_op::DFA_SGID::NUM_SYMBOL_GROUPS, - to_stack_op::DFA_STATES::TT_NUM_STATES>; + using ToStackOpFstT = + cudf::io::fst::detail::Dfa<StackSymbolT, + static_cast<int32_t>( + to_stack_op::DFASymbolGroupID::NUM_SYMBOL_GROUPS), + to_stack_op::DFA_STATES::TT_NUM_STATES>; ToStackOpFstT json_to_stack_ops_fst{to_stack_op::symbol_groups, to_stack_op::transition_table, to_stack_op::translation_table, @@ -361,7 +570,7 @@ void get_stack_context(device_span<SymbolT const> d_json_in, d_stack_ops.data(), device_span<SymbolOffsetT>{d_stack_op_indices.data(), d_stack_op_indices.size()}, JSONToStackOp{}, - d_top_of_stack.data(), + d_top_of_stack, root_symbol, read_symbol, d_json_in.size(), @@ -369,8 +578,8 @@ void get_stack_context(device_span<SymbolT const> d_json_in, } void get_token_stream(device_span<SymbolT const> d_json_in, - device_span<PdaTokenT> d_tokens, - device_span<SymbolOffsetT> d_tokens_indices, + PdaTokenT* d_tokens, + SymbolOffsetT* d_tokens_indices, SymbolOffsetT* d_num_written_tokens, rmm::cuda_stream_view stream) { @@ -378,7 +587,7 @@ void get_token_stream(device_span<SymbolT const> d_json_in, rmm::device_uvector<StackSymbolT> d_top_of_stack{d_json_in.size(), stream}; // Identify what is the stack context for each input character (is it: JSON-root, struct, or list) - get_stack_context(d_json_in, d_top_of_stack, stream); + get_stack_context(d_json_in, d_top_of_stack.data(), stream); // Prepare for PDA transducer pass, merging input symbols with stack symbols rmm::device_uvector<PdaSymbolGroupIdT> d_pda_sgids{d_json_in.size(), stream}; @@ -402,8 +611,8 @@ void get_token_stream(device_span<SymbolT const> d_json_in, // Perform a PDA-transducer pass json_to_tokens_fst.Transduce(d_pda_sgids.begin(), static_cast<SymbolOffsetT>(d_json_in.size()), - d_tokens.data(), - d_tokens_indices.data(), + d_tokens, + d_tokens_indices, d_num_written_tokens, tokenizer_pda::start_state, stream); diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 5a0e39d32b5..ffda2ebf2e5 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -61,10 +61,7 @@ TEST_F(JsonTest, StackContext) d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); // Run algorithm - cudf::io::json::gpu::detail::get_stack_context( - d_input, - cudf::device_span<StackSymbolT>{stack_context.device_ptr(), stack_context.size()}, - stream); + cudf::io::json::gpu::detail::get_stack_context(d_input, stack_context.device_ptr(), stream); // Copy back the results stack_context.device_to_host(stream); @@ -135,12 +132,11 @@ TEST_F(JsonTest, TokenStream) hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream}; // Parse the JSON and get the token stream - cudf::io::json::gpu::detail::get_token_stream( - d_input, - cudf::device_span<PdaTokenT>{tokens_gpu.device_ptr(), tokens_gpu.size()}, - cudf::device_span<SymbolOffsetT>{token_indices_gpu.device_ptr(), token_indices_gpu.size()}, - num_tokens_out.device_ptr(), - stream); + cudf::io::json::gpu::detail::get_token_stream(d_input, + tokens_gpu.device_ptr(), + token_indices_gpu.device_ptr(), + num_tokens_out.device_ptr(), + stream); // Copy back the number of tokens that were written num_tokens_out.device_to_host(stream); @@ -151,30 +147,31 @@ TEST_F(JsonTest, TokenStream) stream_view.synchronize(); // Golden token stream sample + using token_t = nested_json::token_t; std::vector<std::pair<std::size_t, nested_json::PdaTokenT>> golden_token_stream = { - {2, nested_json::TK_BOL}, {3, nested_json::TK_BOS}, {4, nested_json::TK_BFN}, - {13, nested_json::TK_EFN}, {16, nested_json::TK_BST}, {26, nested_json::TK_EST}, - {28, nested_json::TK_BFN}, {35, nested_json::TK_EFN}, {38, nested_json::TK_BOL}, - {39, nested_json::TK_BOV}, {40, nested_json::TK_POV}, {41, nested_json::TK_BOV}, - {43, nested_json::TK_POV}, {44, nested_json::TK_BOV}, {46, nested_json::TK_POV}, - {46, nested_json::TK_EOL}, {48, nested_json::TK_BFN}, {55, nested_json::TK_EFN}, - {58, nested_json::TK_BST}, {69, nested_json::TK_EST}, {71, nested_json::TK_BFN}, - {77, nested_json::TK_EFN}, {80, nested_json::TK_BST}, {105, nested_json::TK_EST}, - {107, nested_json::TK_BFN}, {113, nested_json::TK_EFN}, {116, nested_json::TK_BOV}, - {120, nested_json::TK_POV}, {120, nested_json::TK_EOS}, {124, nested_json::TK_BOS}, - {125, nested_json::TK_BFN}, {134, nested_json::TK_EFN}, {137, nested_json::TK_BST}, - {147, nested_json::TK_EST}, {149, nested_json::TK_BFN}, {155, nested_json::TK_EFN}, - {158, nested_json::TK_BOL}, {159, nested_json::TK_BOV}, {160, nested_json::TK_POV}, - {161, nested_json::TK_BOS}, {162, nested_json::TK_EOS}, {164, nested_json::TK_BOV}, - {168, nested_json::TK_POV}, {169, nested_json::TK_BOS}, {170, nested_json::TK_BFN}, - {172, nested_json::TK_EFN}, {174, nested_json::TK_BOL}, {175, nested_json::TK_BOS}, - {177, nested_json::TK_EOS}, {180, nested_json::TK_BOS}, {181, nested_json::TK_EOS}, - {182, nested_json::TK_EOL}, {184, nested_json::TK_EOS}, {186, nested_json::TK_EOL}, - {188, nested_json::TK_BFN}, {195, nested_json::TK_EFN}, {198, nested_json::TK_BST}, - {209, nested_json::TK_EST}, {211, nested_json::TK_BFN}, {217, nested_json::TK_EFN}, - {220, nested_json::TK_BST}, {252, nested_json::TK_EST}, {254, nested_json::TK_BFN}, - {260, nested_json::TK_EFN}, {263, nested_json::TK_BOV}, {267, nested_json::TK_POV}, - {267, nested_json::TK_EOS}, {268, nested_json::TK_EOL}}; + {2, token_t::ListBegin}, {3, token_t::StructBegin}, {4, token_t::FieldNameBegin}, + {13, token_t::FieldNameEnd}, {16, token_t::StringBegin}, {26, token_t::StringEnd}, + {28, token_t::FieldNameBegin}, {35, token_t::FieldNameEnd}, {38, token_t::ListBegin}, + {39, token_t::ValueBegin}, {40, token_t::ValueEnd}, {41, token_t::ValueBegin}, + {43, token_t::ValueEnd}, {44, token_t::ValueBegin}, {46, token_t::ValueEnd}, + {46, token_t::ListEnd}, {48, token_t::FieldNameBegin}, {55, token_t::FieldNameEnd}, + {58, token_t::StringBegin}, {69, token_t::StringEnd}, {71, token_t::FieldNameBegin}, + {77, token_t::FieldNameEnd}, {80, token_t::StringBegin}, {105, token_t::StringEnd}, + {107, token_t::FieldNameBegin}, {113, token_t::FieldNameEnd}, {116, token_t::ValueBegin}, + {120, token_t::ValueEnd}, {120, token_t::StructEnd}, {124, token_t::StructBegin}, + {125, token_t::FieldNameBegin}, {134, token_t::FieldNameEnd}, {137, token_t::StringBegin}, + {147, token_t::StringEnd}, {149, token_t::FieldNameBegin}, {155, token_t::FieldNameEnd}, + {158, token_t::ListBegin}, {159, token_t::ValueBegin}, {160, token_t::ValueEnd}, + {161, token_t::StructBegin}, {162, token_t::StructEnd}, {164, token_t::ValueBegin}, + {168, token_t::ValueEnd}, {169, token_t::StructBegin}, {170, token_t::FieldNameBegin}, + {172, token_t::FieldNameEnd}, {174, token_t::ListBegin}, {175, token_t::StructBegin}, + {177, token_t::StructEnd}, {180, token_t::StructBegin}, {181, token_t::StructEnd}, + {182, token_t::ListEnd}, {184, token_t::StructEnd}, {186, token_t::ListEnd}, + {188, token_t::FieldNameBegin}, {195, token_t::FieldNameEnd}, {198, token_t::StringBegin}, + {209, token_t::StringEnd}, {211, token_t::FieldNameBegin}, {217, token_t::FieldNameEnd}, + {220, token_t::StringBegin}, {252, token_t::StringEnd}, {254, token_t::FieldNameBegin}, + {260, token_t::FieldNameEnd}, {263, token_t::ValueBegin}, {267, token_t::ValueEnd}, + {267, token_t::StructEnd}, {268, token_t::ListEnd}}; // Verify the number of tokens matches ASSERT_EQ(golden_token_stream.size(), num_tokens_out[0]); From 0a7821e0f3acc23b314c50f90d3885a3ebbd8673 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 21 Jul 2022 01:57:57 -0700 Subject: [PATCH 53/81] fixes rebase conflicts --- cpp/src/io/fst/logical_stack.cuh | 51 -------------------------- cpp/tests/io/fst/fst_test.cu | 3 -- cpp/tests/io/fst/logical_stack_test.cu | 1 - 3 files changed, 55 deletions(-) diff --git a/cpp/src/io/fst/logical_stack.cuh b/cpp/src/io/fst/logical_stack.cuh index 5149ade7c1c..9502922a379 100644 --- a/cpp/src/io/fst/logical_stack.cuh +++ b/cpp/src/io/fst/logical_stack.cuh @@ -216,35 +216,6 @@ struct RemapEmptyStack { StackOpT empty_stack_symbol; }; -/** - * @brief Function object to return only the stack_level part from a StackOp instance. - */ -struct StackOpToStackLevel { - template <typename StackLevelT, typename ValueT> - constexpr CUDF_HOST_DEVICE StackLevelT operator()(StackOp<StackLevelT, ValueT> const& kv_op) const - { - return kv_op.stack_level; - } -}; - -/** - * @brief Retrieves an iterator that returns only the `stack_level` part from a StackOp iterator. - */ -template <typename StackOpItT> -auto get_stack_level_iterator(StackOpItT it) -{ - return thrust::make_transform_iterator(it, StackOpToStackLevel{}); -} - -/** - * @brief Retrieves an iterator that returns only the `value` part from a StackOp iterator. - */ -template <typename StackOpItT> -auto get_value_iterator(StackOpItT it) -{ - return thrust::make_transform_iterator(it, StackOpToStackSymbol{}); -} - } // namespace detail /** @@ -430,14 +401,6 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, num_symbols_in, stream)); - // Dump info on stack operations: (stack level change + symbol) -> (absolute stack level + symbol) - test::print::print_array(num_symbols_in, - stream, - get_stack_level_iterator(stack_symbols_in), - get_value_iterator(stack_symbols_in), - get_stack_level_iterator(d_kv_operations.Current()), - get_value_iterator(d_kv_operations.Current())); - // Stable radix sort, sorting by stack level of the operations d_kv_operations_unsigned = cub::DoubleBuffer<StackOpUnsignedT>{ reinterpret_cast<StackOpUnsignedT*>(d_kv_operations.Current()), @@ -466,15 +429,6 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, num_symbols_in, stream)); - // Dump info on stack operations sorted by their stack level (i.e. stack level after applying - // operation) - test::print::print_array(num_symbols_in, - stream, - get_stack_level_iterator(kv_ops_scan_in), - get_value_iterator(kv_ops_scan_in), - get_stack_level_iterator(kv_ops_scan_out), - get_value_iterator(kv_ops_scan_out)); - // Fill the output tape with read-symbol thrust::fill(rmm::exec_policy(stream), thrust::device_ptr<StackSymbolT>{d_top_of_stack}, @@ -493,11 +447,6 @@ void sparse_stack_op_to_top_of_stack(StackSymbolItT d_symbols, d_symbol_positions_db.Current(), d_top_of_stack); - // Dump the output tape that has many yet-to-be-filled spots (i.e., all spots that were not given - // in the sparse representation) - test::print::print_array( - std::min(num_symbols_in, static_cast<decltype(num_symbols_in)>(10000)), stream, d_top_of_stack); - // We perform an exclusive scan in order to fill the items at the very left that may // be reading the empty stack before there's the first push occurrence in the sequence. // Also, we're interested in the top-of-the-stack symbol before the operation was applied. diff --git a/cpp/tests/io/fst/fst_test.cu b/cpp/tests/io/fst/fst_test.cu index a668b764793..e198c804222 100644 --- a/cpp/tests/io/fst/fst_test.cu +++ b/cpp/tests/io/fst/fst_test.cu @@ -26,7 +26,6 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> -#include <rmm/cuda_stream.hpp> #include <rmm/device_buffer.hpp> #include <rmm/device_uvector.hpp> @@ -212,8 +211,6 @@ TEST_F(FstTest, GroundTruth) auto& d_input = static_cast<cudf::scalar_type_t<std::string>&>(*d_input_string); input = d_input.to_string(stream); - - // Prepare input & output buffers constexpr std::size_t single_item = 1; hostdevice_vector<SymbolT> output_gpu(input.size(), stream_view); diff --git a/cpp/tests/io/fst/logical_stack_test.cu b/cpp/tests/io/fst/logical_stack_test.cu index 831e9140989..dda737f005d 100644 --- a/cpp/tests/io/fst/logical_stack_test.cu +++ b/cpp/tests/io/fst/logical_stack_test.cu @@ -22,7 +22,6 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> -#include <rmm/device_buffer.hpp> #include <rmm/device_uvector.hpp> #include <cstdlib> From 7396335e667dbe4932020577dc2ffe491d063e3a Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 21 Jul 2022 02:56:48 -0700 Subject: [PATCH 54/81] fixes escape sequence inside strings and field names and adds test for it --- cpp/src/io/json/nested_json_gpu.cu | 5 ++-- cpp/tests/io/nested_json_test.cu | 42 ++++++++++++++++-------------- 2 files changed, 25 insertions(+), 22 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 07994ea2fa7..d1026396d10 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -77,7 +77,7 @@ const std::vector<std::string> symbol_groups = {"{", "[", "}", "]", "\"", "\\"}; const std::vector<std::vector<StateT>> transition_table = { /* IN_STATE { [ } ] " \ OTHER */ /* TT_OOS */ {TT_OOS, TT_OOS, TT_OOS, TT_OOS, TT_STR, TT_OOS, TT_OOS}, - /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_STR, TT_STR}, + /* TT_STR */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_OOS, TT_ESC, TT_STR}, /* TT_ESC */ {TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR, TT_STR}}; // Translation table (i.e., for each transition, what are the symbols that we output) @@ -158,7 +158,8 @@ static __constant__ PdaSymbolGroupIdT tos_sg_to_pda_sgid[] = { OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OTR, OBC, OTR, CBC, OTR}; /** - * @brief Maps a (top-of-stack symbol, input symbol)-pair to a symbol group id of the DVPA + * @brief Maps a (top-of-stack symbol, input symbol)-pair to a symbol group id of the deterministic + * visibly pushdown automaton (DVPA) */ struct PdaSymbolToSymbolGroupId { template <typename SymbolT, typename StackSymbolT> diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index ffda2ebf2e5..90afb9dd654 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -49,7 +49,7 @@ TEST_F(JsonTest, StackContext) R"("category": "reference",)" R"("index": [4,{},null,{"a":[{ }, {}] } ],)" R"("author": "Nigel Rees",)" - R"("title": "{}[], <=semantic-symbols-string",)" + R"("title": "{}\\\"[], <=semantic-symbols-string\\\"",)" R"("price": 8.95)" R"(}] )"; @@ -70,25 +70,25 @@ TEST_F(JsonTest, StackContext) stream_view.synchronize(); std::vector<char> golden_stack_context{ - '_', '_', '_', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '[', '[', '[', '[', '[', '[', '[', '[', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', '[', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', - '{', '[', '[', '[', '[', '[', '[', '[', '{', '{', '{', '{', '{', '[', '{', '{', '[', '[', - '[', '{', '[', '{', '{', '[', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '_'}; + '_', '_', '_', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '[', '[', '[', '[', '[', '[', '[', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', '[', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', '{', '[', '[', '[', '[', '[', '[', '[', '{', + '{', '{', '{', '{', '[', '{', '{', '[', '[', '[', '{', '[', '{', '{', '[', '[', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', + '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '_'}; ASSERT_EQ(golden_stack_context.size(), stack_context.size()); - for (std::size_t i = 0; i < stack_context.size() && i < 1000; i++) { - ASSERT_EQ(golden_stack_context[i], stack_context[i]); + for (std::size_t i = 0; i < stack_context.size(); i++) { + EXPECT_EQ(golden_stack_context[i], stack_context[i]) << "Mismatch at #" << i; } } @@ -178,8 +178,10 @@ TEST_F(JsonTest, TokenStream) for (std::size_t i = 0; i < num_tokens_out[0]; i++) { // Ensure the index the tokens are pointing to do match - ASSERT_EQ(golden_token_stream[i].first, token_indices_gpu[i]); + EXPECT_EQ(golden_token_stream[i].first, token_indices_gpu[i]) << "Mismatch at #" << i; + ; // Ensure the token category is correct - ASSERT_EQ(golden_token_stream[i].second, tokens_gpu[i]); + EXPECT_EQ(golden_token_stream[i].second, tokens_gpu[i]) << "Mismatch at #" << i; + ; } } From 6252208e06eaa6ace4a57329cee46aa7d8a68122 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 21 Jul 2022 04:54:53 -0700 Subject: [PATCH 55/81] adds comments on pda transition table states --- cpp/src/io/json/nested_json_gpu.cu | 20 +++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index d1026396d10..b903e386f70 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -179,26 +179,39 @@ struct PdaSymbolToSymbolGroupId { (stack_symbol == '_') ? STACK_ROOT : ((stack_symbol == '[') ? STACK_LIST : STACK_STRUCT); // The relative symbol group id of the current input symbol - PdaSymbolGroupIdT symbol_gid = tos_sg_to_pda_sgid[min( - static_cast<int32_t>(symbol), - static_cast<int32_t>(sizeof(tos_sg_to_pda_sgid) / sizeof(tos_sg_to_pda_sgid[0])) - 1)]; + constexpr int32_t pda_sgid_lookup_size = + static_cast<int32_t>(sizeof(tos_sg_to_pda_sgid) / sizeof(tos_sg_to_pda_sgid[0])); + PdaSymbolGroupIdT symbol_gid = + tos_sg_to_pda_sgid[min(static_cast<int32_t>(symbol), pda_sgid_lookup_size - 1)]; return stack_idx * NUM_PDA_INPUT_SGS + symbol_gid; } }; // The states defined by the pushdown automaton enum pda_state_t : StateT { + // Beginning of value PD_BOV, + // Beginning of array PD_BOA, + // Literal or number PD_LON, + // String PD_STR, + // After escape char when within string PD_SCE, + // After having parsed a value PD_PVL, + // Before the next field name PD_BFN, + // Field name PD_FLN, + // After escape char when within field name PD_FNE, + // After a field name inside a struct PD_PFN, + // Error state (trap state) PD_ERR, + // Total number of PDA states PD_NUM_STATES }; @@ -216,6 +229,7 @@ const std::vector<std::vector<char>> pda_sgids{ std::vector<std::vector<StateT>> get_transition_table() { std::vector<std::vector<StateT>> pda_tt(PD_NUM_STATES); + // { [ } ] " \ , : space other pda_tt[PD_BOV] = {PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON, PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON, PD_BOA, PD_BOA, PD_ERR, PD_ERR, PD_STR, PD_ERR, PD_ERR, PD_ERR, PD_BOV, PD_LON}; From 191d71d00b04e56ddf26ce55294d2f44c5bae28a Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Fri, 22 Jul 2022 04:33:25 -0700 Subject: [PATCH 56/81] adopts new verification macro in test --- cpp/tests/io/nested_json_test.cu | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 90afb9dd654..01d90b44690 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -49,7 +49,7 @@ TEST_F(JsonTest, StackContext) R"("category": "reference",)" R"("index": [4,{},null,{"a":[{ }, {}] } ],)" R"("author": "Nigel Rees",)" - R"("title": "{}\\\"[], <=semantic-symbols-string\\\"",)" + R"("title": "{}\\\"[], <=semantic-symbols-string\\\\",)" R"("price": 8.95)" R"(}] )"; @@ -87,9 +87,7 @@ TEST_F(JsonTest, StackContext) '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '_'}; ASSERT_EQ(golden_stack_context.size(), stack_context.size()); - for (std::size_t i = 0; i < stack_context.size(); i++) { - EXPECT_EQ(golden_stack_context[i], stack_context[i]) << "Mismatch at #" << i; - } + CUDF_TEST_EXPECT_VECTOR_EQUAL(golden_stack_context, stack_context, stack_context.size()); } TEST_F(JsonTest, TokenStream) From 3b9a1ed174cc6bd6511aaa83b57097a038447f1b Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Fri, 22 Jul 2022 06:46:17 -0700 Subject: [PATCH 57/81] removes superfluous semicolons --- cpp/tests/io/nested_json_test.cu | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 01d90b44690..0a04059a393 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -177,9 +177,8 @@ TEST_F(JsonTest, TokenStream) for (std::size_t i = 0; i < num_tokens_out[0]; i++) { // Ensure the index the tokens are pointing to do match EXPECT_EQ(golden_token_stream[i].first, token_indices_gpu[i]) << "Mismatch at #" << i; - ; + // Ensure the token category is correct EXPECT_EQ(golden_token_stream[i].second, tokens_gpu[i]) << "Mismatch at #" << i; - ; } } From 632be35f4bf9a7ba89141d6b4cb6d42834e00798 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Sat, 23 Jul 2022 06:41:27 -0700 Subject: [PATCH 58/81] rearranges token order in enum and adds documentation --- cpp/src/io/json/nested_json.h | 31 +++++++++++++++++-------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/cpp/src/io/json/nested_json.h b/cpp/src/io/json/nested_json.h index 6bc99d50bd8..7480c1eed9b 100644 --- a/cpp/src/io/json/nested_json.h +++ b/cpp/src/io/json/nested_json.h @@ -40,7 +40,7 @@ using PdaInputSymbolGroupIdT = char; /// Type used to represent a symbol group id of the stack alphabet in the pushdown automaton using PdaStackSymbolGroupIdT = char; -/// Type used to represent a (input-symbol, stack-symbole)-tuple in stack-symbole-major order +/// Type used to represent a (input-symbol, stack-symbol)-tuple in stack-symbol-major order using PdaSymbolGroupIdT = char; /// Type being emitted by the pushdown automaton transducer @@ -52,26 +52,26 @@ using PdaTokenT = char; enum token_t : PdaTokenT { /// Beginning-of-struct token (on encounter of semantic '{') StructBegin, + /// End-of-struct token (on encounter of semantic '}') + StructEnd, /// Beginning-of-list token (on encounter of semantic '[') ListBegin, - /// Beginning-of-error token (on first encounter of a parsing error) - ErrorBegin, - /// Beginning-of-string-value token (on encounter of the string's first quote) - StringBegin, - /// Beginning-of-value token (first character of literal or numeric) - ValueBegin, /// End-of-list token (on encounter of semantic ']') ListEnd, - /// End-of-struct token (on encounter of semantic '}') - StructEnd, /// Beginning-of-field-name token (on encounter of first quote) FieldNameBegin, - /// Post-value token (first character after a literal or numeric string) - ValueEnd, - /// End-of-string token (on encounter of a string's second quote) - StringEnd, /// End-of-field-name token (on encounter of a field name's second quote) FieldNameEnd, + /// Beginning-of-string-value token (on encounter of the string's first quote) + StringBegin, + /// End-of-string token (on encounter of a string's second quote) + StringEnd, + /// Beginning-of-value token (first character of literal or numeric) + ValueBegin, + /// Post-value token (first character after a literal or numeric string) + ValueEnd, + /// Beginning-of-error token (on first encounter of a parsing error) + ErrorBegin, /// Total number of tokens NUM_TOKENS }; @@ -84,7 +84,10 @@ namespace detail { * bracket would actually pop a the corresponding opening brace. * * @param[in] d_json_in The string of input characters - * @param[out] d_top_of_stack + * @param[out] d_top_of_stack Will be populated with what-is-on-top-of-the-stack for any given input + * character of \p d_json_in, where a '{' represents that the corresponding input character is + * within the context of a struct, a '[' represents that it is within the context of an array, and a + * '_' symbol that it is at the root of the JSON. * @param[in] stream The cuda stream to dispatch GPU kernels to */ void get_stack_context(device_span<SymbolT const> d_json_in, From 32377723109ad5e87cf2b7dad2d4c54995d27477 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Sat, 23 Jul 2022 06:42:05 -0700 Subject: [PATCH 59/81] uses namespace alias and switch to rmm stream in test --- cpp/tests/io/nested_json_test.cu | 55 ++++++++++++++++++-------------- 1 file changed, 31 insertions(+), 24 deletions(-) diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 0a04059a393..c829160ce00 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -20,6 +20,9 @@ #include <cudf_test/base_fixture.hpp> #include <cudf_test/cudf_gtest.hpp> +#include <rmm/cuda_stream.hpp> +#include <rmm/cuda_stream_view.hpp> + namespace nested_json = cudf::io::json::gpu; // Base test fixture for tests @@ -33,8 +36,7 @@ TEST_F(JsonTest, StackContext) using StackSymbolT = char; // Prepare cuda stream for data transfers & kernels - cudaStream_t stream = nullptr; - cudaStreamCreate(&stream); + rmm::cuda_stream stream{}; rmm::cuda_stream_view stream_view(stream); // Test input @@ -57,14 +59,17 @@ TEST_F(JsonTest, StackContext) rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); hostdevice_vector<StackSymbolT> stack_context(input.size(), stream_view); - ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( - d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); + ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync(d_input.data(), + input.data(), + input.size() * sizeof(SymbolT), + cudaMemcpyHostToDevice, + stream.value())); // Run algorithm - cudf::io::json::gpu::detail::get_stack_context(d_input, stack_context.device_ptr(), stream); + nested_json::detail::get_stack_context(d_input, stack_context.device_ptr(), stream_view); // Copy back the results - stack_context.device_to_host(stream); + stack_context.device_to_host(stream_view); // Make sure we copied back the stack context stream_view.synchronize(); @@ -92,15 +97,14 @@ TEST_F(JsonTest, StackContext) TEST_F(JsonTest, TokenStream) { - using cudf::io::json::gpu::PdaTokenT; - using cudf::io::json::gpu::SymbolOffsetT; - using cudf::io::json::gpu::SymbolT; + using nested_json::PdaTokenT; + using nested_json::SymbolOffsetT; + using nested_json::SymbolT; constexpr std::size_t single_item = 1; // Prepare cuda stream for data transfers & kernels - cudaStream_t stream = nullptr; - cudaStreamCreate(&stream); + rmm::cuda_stream stream{}; rmm::cuda_stream_view stream_view(stream); // Test input @@ -122,24 +126,27 @@ TEST_F(JsonTest, TokenStream) // Prepare input & output buffers rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); - ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync( - d_input.data(), input.data(), input.size() * sizeof(SymbolT), cudaMemcpyHostToDevice, stream)); + ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync(d_input.data(), + input.data(), + input.size() * sizeof(SymbolT), + cudaMemcpyHostToDevice, + stream.value())); - hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream}; - hostdevice_vector<SymbolOffsetT> token_indices_gpu{input.size(), stream}; - hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream}; + hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream_view}; + hostdevice_vector<SymbolOffsetT> token_indices_gpu{input.size(), stream_view}; + hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream_view}; // Parse the JSON and get the token stream - cudf::io::json::gpu::detail::get_token_stream(d_input, - tokens_gpu.device_ptr(), - token_indices_gpu.device_ptr(), - num_tokens_out.device_ptr(), - stream); + nested_json::detail::get_token_stream(d_input, + tokens_gpu.device_ptr(), + token_indices_gpu.device_ptr(), + num_tokens_out.device_ptr(), + stream_view); // Copy back the number of tokens that were written - num_tokens_out.device_to_host(stream); - tokens_gpu.device_to_host(stream); - token_indices_gpu.device_to_host(stream); + num_tokens_out.device_to_host(stream_view); + tokens_gpu.device_to_host(stream_view); + token_indices_gpu.device_to_host(stream_view); // Make sure we copied back all relevant data stream_view.synchronize(); From d2832b994134996197470bad13075b7b34302a6b Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Sat, 23 Jul 2022 22:04:32 -0700 Subject: [PATCH 60/81] drops the gpu namespace --- cpp/src/io/json/nested_json.h | 4 ++-- cpp/src/io/json/nested_json_gpu.cu | 4 ++-- cpp/tests/io/nested_json_test.cu | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cpp/src/io/json/nested_json.h b/cpp/src/io/json/nested_json.h index 7480c1eed9b..cb45acaa60e 100644 --- a/cpp/src/io/json/nested_json.h +++ b/cpp/src/io/json/nested_json.h @@ -20,7 +20,7 @@ #include <rmm/cuda_stream_view.hpp> -namespace cudf::io::json::gpu { +namespace cudf::io::json { /// Type used to represent the atomic symbol type used within the finite-state machine using SymbolT = char; @@ -112,4 +112,4 @@ void get_token_stream(device_span<SymbolT const> d_json_in, rmm::cuda_stream_view stream); } // namespace detail -} // namespace cudf::io::json::gpu +} // namespace cudf::io::json diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index b903e386f70..955cff4c2fb 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -25,7 +25,7 @@ #include <rmm/exec_policy.hpp> -namespace cudf::io::json::gpu { +namespace cudf::io::json { //------------------------------------------------------------------------------ // JSON-TO-STACK-OP DFA @@ -635,4 +635,4 @@ void get_token_stream(device_span<SymbolT const> d_json_in, } // namespace detail -} // namespace cudf::io::json::gpu +} // namespace cudf::io::json diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index c829160ce00..c3a9d2d1c1f 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -23,7 +23,7 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> -namespace nested_json = cudf::io::json::gpu; +namespace nested_json = cudf::io::json; // Base test fixture for tests struct JsonTest : public cudf::test::BaseFixture { From 618ed3f63ce21c4216eb013aa9da2e3d983ae72e Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Sat, 23 Jul 2022 22:05:43 -0700 Subject: [PATCH 61/81] renames header file extension from h to hpp --- cpp/src/io/json/{nested_json.h => nested_json.hpp} | 0 cpp/src/io/json/nested_json_gpu.cu | 2 +- cpp/tests/io/nested_json_test.cu | 2 +- 3 files changed, 2 insertions(+), 2 deletions(-) rename cpp/src/io/json/{nested_json.h => nested_json.hpp} (100%) diff --git a/cpp/src/io/json/nested_json.h b/cpp/src/io/json/nested_json.hpp similarity index 100% rename from cpp/src/io/json/nested_json.h rename to cpp/src/io/json/nested_json.hpp diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 955cff4c2fb..70294706461 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -14,7 +14,7 @@ * limitations under the License. */ -#include "nested_json.h" +#include "nested_json.hpp" #include <io/fst/logical_stack.cuh> #include <io/fst/lookup_tables.cuh> diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index c3a9d2d1c1f..7d74a33adc4 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -14,7 +14,7 @@ * limitations under the License. */ -#include <io/json/nested_json.h> +#include <io/json/nested_json.hpp> #include <io/utilities/hostdevice_vector.hpp> #include <cudf_test/base_fixture.hpp> From 19b37b727e4fd8f4236f79e6906801eb530da3ba Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 27 Jul 2022 07:10:05 -0700 Subject: [PATCH 62/81] squashed with minimal example --- cpp/src/io/json/nested_json.hpp | 94 ++++++ cpp/src/io/json/nested_json_gpu.cu | 509 ++++++++++++++++++++++++++++- cpp/tests/io/nested_json_test.cu | 126 +++++++ 3 files changed, 728 insertions(+), 1 deletion(-) diff --git a/cpp/src/io/json/nested_json.hpp b/cpp/src/io/json/nested_json.hpp index cb45acaa60e..e0cd2de9286 100644 --- a/cpp/src/io/json/nested_json.hpp +++ b/cpp/src/io/json/nested_json.hpp @@ -16,10 +16,16 @@ #pragma once +#include <cudf/io/types.hpp> + +#include <cudf/types.hpp> #include <cudf/utilities/span.hpp> #include <rmm/cuda_stream_view.hpp> +#include <variant> +#include <vector> + namespace cudf::io::json { /// Type used to represent the atomic symbol type used within the finite-state machine @@ -46,6 +52,84 @@ using PdaSymbolGroupIdT = char; /// Type being emitted by the pushdown automaton transducer using PdaTokenT = char; +/// Type used to represent the class of a node (or a node "category") within the tree representation +using NodeT = char; + +/// Type used to index into the nodes within the tree of structs, lists, field names, and value +/// nodes +using NodeIndexT = uint32_t; + +/// Type large enough to represent tree depth from [0, max-tree-depth); may be an unsigned type +using TreeDepthT = StackLevelT; + +/** + * @brief Struct that encapsulate all information of a columnar tree representation. + */ +struct tree_meta_t { + std::vector<NodeT> node_categories; + std::vector<NodeIndexT> parent_node_ids; + std::vector<TreeDepthT> node_levels; + std::vector<SymbolOffsetT> node_range_begin; + std::vector<SymbolOffsetT> node_range_end; +}; + +constexpr NodeIndexT parent_node_sentinel = std::numeric_limits<NodeIndexT>::max(); + +/** + * @brief Class of a node (or a node "category") within the tree representation + */ +enum node_t : NodeT { + /// A node representing a struct + NC_STRUCT, + /// A node representing a list + NC_LIST, + /// A node representing a field name + NC_FN, + /// A node representing a string value + NC_STR, + /// A node representing a numeric or literal value (e.g., true, false, null) + NC_VAL, + /// A node representing a parser error + NC_ERR, + /// Total number of node classes + NUM_NODE_CLASSES +}; + +/** + * @brief A column type + */ +enum class json_col_t : char { ListColumn, StructColumn, StringColumn, Unknown }; + +/** + * @brief + * + */ +struct json_column { + // Type used to count number of rows + using row_offset_t = uint32_t; + + // The inferred type of this column (list, struct, or value/string column) + json_col_t type = json_col_t::Unknown; + + std::vector<row_offset_t> string_offsets; + std::vector<row_offset_t> string_lengths; + + // Row offsets + std::vector<row_offset_t> child_offsets; + + // Validity bitmap + std::vector<bool> validity; + row_offset_t valid_count = 0; + + // Map of child columns, if applicable. + // Following "items" as the default child column's name of a list column + // Using the struct's field names + std::map<std::string, json_column> child_columns; + + // Counting the current number of items in this column + row_offset_t current_offset = 0; +}; + /** * @brief Tokens emitted while parsing a JSON input */ @@ -110,6 +194,16 @@ void get_token_stream(device_span<SymbolT const> d_json_in, SymbolOffsetT* d_tokens_indices, SymbolOffsetT* d_num_written_tokens, rmm::cuda_stream_view stream); + +/** + * @brief Parses the given JSON string and generates a tree representation of the given input. + * + * @param input The JSON input + * @param stream The CUDA stream to which kernels are dispatched + * @return + */ +json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream); + } // namespace detail } // namespace cudf::io::json diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 70294706461..131522c8c66 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -14,6 +14,7 @@ * limitations under the License. */ +#include "cudf/utilities/error.hpp" #include "nested_json.hpp" #include <io/fst/logical_stack.cuh> @@ -23,8 +24,11 @@ #include <cudf/types.hpp> #include <cudf/utilities/span.hpp> +#include <iterator> #include <rmm/exec_policy.hpp> +#include <stack> + namespace cudf::io::json { //------------------------------------------------------------------------------ @@ -633,6 +637,509 @@ void get_token_stream(device_span<SymbolT const> d_json_in, stream); } -} // namespace detail +struct tree_node { + // The column that this node is associated with + // E.g., if this is a struct node, it's the struct's column + json_column* column; + + // The row offset that this node belongs to within the given column + uint32_t row_index; + + // Selected child column + // E.g., if this is a struct node, and we subsequently encountered the field name "a", then this + // point's to the struct's "a" child column + json_column* current_selected_col = nullptr; + + std::size_t num_children = 0; +}; + +/** + * RULES: + * + * if first node type of a column is: + * --> a list: it becomes list column + * --> a struct: it becomes struct column + * --> else (e.g., 'null', '123', '"foo"'): it becomes a string column + */ +json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) +{ + // Default name for a list's child column + std::string const list_child_name = "items"; + + constexpr std::size_t single_item = 1; + hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream}; + hostdevice_vector<SymbolOffsetT> token_indices_gpu{input.size(), stream}; + hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream}; + + // Allocate device memory for the JSON input & copy over to device + rmm::device_uvector<SymbolT> d_input{input.size(), stream}; + cudaMemcpyAsync( + d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); + + // Parse the JSON and get the token stream + get_token_stream(cudf::device_span<SymbolT>{d_input.data(), d_input.size()}, + tokens_gpu.device_ptr(), + token_indices_gpu.device_ptr(), + num_tokens_out.device_ptr(), + stream); + + // Copy the JSON tokens to the host + token_indices_gpu.device_to_host(stream); + tokens_gpu.device_to_host(stream); + num_tokens_out.device_to_host(stream); + + // Make sure tokens have been copied to the host + stream.synchronize(); + + // Whether this token is the valid token to begin the JSON document with + auto is_valid_root_token = [](PdaTokenT const token) { + switch (token) { + case token_t::StructBegin: + case token_t::ListBegin: + case token_t::StringBegin: + case token_t::ValueBegin: return true; + default: return false; + }; + }; + + // Returns the token's corresponding column type + auto token_to_column_type = [](PdaTokenT const token) { + switch (token) { + case token_t::StructBegin: return json_col_t::StructColumn; + case token_t::ListBegin: return json_col_t::ListColumn; + case token_t::StringBegin: return json_col_t::StringColumn; + case token_t::ValueBegin: return json_col_t::StringColumn; + default: return json_col_t::Unknown; + }; + }; + + // Whether this token is a beginning-of-list or beginning-of-struct token + auto is_nested_token = [](PdaTokenT const token) { + switch (token) { + case token_t::StructBegin: + case token_t::ListBegin: return true; + default: return false; + }; + }; + + // Skips the quote char if the token is a beginning-of-string or beginning-of-field-name token + auto get_token_index = [](PdaTokenT const token, SymbolOffsetT const token_index) { + constexpr SymbolOffsetT skip_quote_char = 1; + switch (token) { + case token_t::StringBegin: return token_index + skip_quote_char; + case token_t::FieldNameBegin: return token_index + skip_quote_char; + default: return token_index; + }; + }; + + // The end-of-* partner token for a given beginning-of-* token + auto end_of_partner = [](PdaTokenT const token) { + switch (token) { + case token_t::StringBegin: return token_t::StringEnd; + case token_t::ValueBegin: return token_t::ValueEnd; + case token_t::FieldNameBegin: return token_t::FieldNameEnd; + default: return token_t::ErrorBegin; + }; + }; + + auto column_type_string = [](json_col_t column_type) { + switch (column_type) { + case json_col_t::Unknown: return "Unknown"; + case json_col_t::ListColumn: return "List"; + case json_col_t::StructColumn: return "Struct"; + case json_col_t::StringColumn: return "String"; + default: return "Unknown"; + } + }; + + auto append_row_to_column = [&input, &column_type_string](json_column* column, + uint32_t row_index, + json_col_t const& row_type, + uint32_t string_offset, + uint32_t string_end, + uint32_t child_count) { + + + CUDF_EXPECTS(column != nullptr, "Encountered invalid JSON token sequence"); + + std::cout << " -> append_row_to_column()\n"; + std::cout << " ---> col@" << column << "\n"; + std::cout << " ---> row #" << row_index << "\n"; + std::cout << " ---> col.type: " << column_type_string(column->type) << "\n"; + std::cout << " ---> row_type: " << column_type_string(row_type) << "\n"; + std::cout << " ---> string: '" + << std::string{input.data() + string_offset, (string_end - string_offset)} << "'\n"; + + // If, thus far, the column's type couldn't be inferred, we infer it to the given type + if (column->type == json_col_t::Unknown) { column->type = row_type; } + + // We shouldn't run into this, as we shouldn't + CUDF_EXPECTS(column->type != json_col_t::Unknown, "Encountered invalid JSON token sequence"); + + // Check for the last inserted row offset + auto populated_row_count = column->current_offset; + if (populated_row_count < row_index) { + + std::cout << " ---> filling [" << populated_row_count << ", " << row_index << ")\n"; + } + + // Fill all the omitted rows with "empty"/null rows + std::fill_n(std::back_inserter(column->validity), row_index - column->string_offsets.size(), false); + std::fill_n(std::back_inserter(column->string_offsets), + row_index - column->string_offsets.size(), + (column->string_offsets.size() > 0) ? column->string_offsets.back() : 0); + std::fill_n( + std::back_inserter(column->string_lengths), row_index - column->string_lengths.size(), 0); + std::fill_n(std::back_inserter(column->child_offsets), + row_index + 1 - column->child_offsets.size(), + (column->child_offsets.size() > 0) ? column->child_offsets.back() : 0); + + // Fill all the omitted rows with "empty" rows + // col type | row type => + // List | List => valid + // List | Struct => FAIL + // List | String => null + // Struct | List => FAIL + // Struct | Struct => valid + // Struct | String => null + // String | List => null + // String | Struct => null + // String | String => valid + bool is_valid = (column->type == row_type); + column->validity.push_back(is_valid); + column->valid_count += (is_valid) ? 1U : 0U; + column->string_offsets.push_back(string_offset); + column->string_lengths.push_back(string_end - string_offset); + column->child_offsets.push_back( + (column->child_offsets.size() > 0) ? column->child_offsets.back() + child_count : 0); + column->current_offset++; + }; + + auto update_row = + [](json_column* column, uint32_t row_index, uint32_t string_end, uint32_t child_count) { + std::cout << " -> update_row()\n"; + std::cout << " ---> col@" << column << "\n"; + std::cout << " ---> row #" << row_index << "\n"; + std::cout << " ---> string_lengths = " << (string_end - column->string_offsets[row_index]) + << "\n"; + std::cout << " ---> child_offsets = " << (column->child_offsets[row_index + 1] + child_count) + << "\n"; + column->string_lengths[row_index] = column->child_offsets[row_index + 1] + child_count; + column->child_offsets[row_index + 1] = column->child_offsets[row_index + 1] + child_count; + }; + + /** + * @brief Gets the currently selected child column given a \p current_data_path. + * + * That is, if \p current_data_path top-of-stack is + * (a) a struct, the selected child column corresponds to the child column of the last field name + * node encoutnered. + * (b) a list, the selected child column corresponds to single child column of + * the list column. In this case, the child column may not exist yet. + * + */ + auto get_selected_column = [&list_child_name](std::stack<tree_node>& current_data_path) { + std::cout << " -> get_selected_column()\n"; + json_column* selected_col = current_data_path.top().current_selected_col; + + // If the node does not have a selected column yet + if (selected_col == nullptr) { + // We're looking at the child column of a list column + if (current_data_path.top().column->type == json_col_t::ListColumn) { + CUDF_EXPECTS(current_data_path.top().column->child_columns.size() <= 1, + "Encountered a list column with more than a single child column"); + // The child column has yet to be created + if (current_data_path.top().column->child_columns.size() == 0) { + current_data_path.top().column->child_columns.insert( + {list_child_name, {json_col_t::Unknown}}); + } + current_data_path.top().current_selected_col = + ¤t_data_path.top().column->child_columns.begin()->second; + std::cout << " ---> selected col@" << current_data_path.top().current_selected_col << "\n"; + return current_data_path.top().current_selected_col; + } else { + CUDF_FAIL("Trying to retrieve child column without encountering a field name."); + } + } else { + std::cout << " ---> selected col@" << selected_col << "\n"; + return selected_col; + } + }; + + auto select_column = [](std::stack<tree_node>& current_data_path, std::string const& field_name) { + std::cout << " -> select_column(" << field_name << ")\n"; + // The field name's parent struct node + auto& current_struct_node = current_data_path.top(); + + // Verify that the field name node is actually a child of a struct + CUDF_EXPECTS(current_data_path.top().column->type == json_col_t::StructColumn, + "Invalid JSON token sequence"); + + json_column* struct_col = current_struct_node.column; + auto const& child_col_it = struct_col->child_columns.find(field_name); + + // The field name's column exists already, select that as the struct node's currently selected + // child column + if (child_col_it != struct_col->child_columns.end()) { + std::cout << " ---> selected col@" << &child_col_it->second << "\n"; + return &child_col_it->second; + } + + // The field name's column does not exist yet, so we have to append the child column to the + // struct column + json_column* ptr = &struct_col->child_columns.insert({field_name, {}}).first->second; + json_column* ptr2 = &struct_col->child_columns.find(field_name)->second; + std::cout << "ptr1: " << static_cast<void*>(ptr) << ", ptr2: " << static_cast<void*>(ptr2) + << "\n"; + return &struct_col->child_columns.insert({field_name, {}}).first->second; + }; + + // The stack represents the path from the JSON root node to the current node being looked at + std::stack<tree_node> current_data_path; + + // The offset of the token currently being processed + std::size_t offset = 0; + + // The root column + json_column root_column; + + // Giving names to magic constants + constexpr uint32_t row_offset_zero = 0; + constexpr uint32_t zero_child_count = 0; + + //-------------------------------------------------------------------------------- + // INITIALIZE JSON ROOT NODE + //-------------------------------------------------------------------------------- + // The JSON root may only a struct, list, string, or value node + CUDF_EXPECTS(num_tokens_out[0] > 0, "Empty JSON input not supported"); + CUDF_EXPECTS(is_valid_root_token(tokens_gpu[offset]), "Invalid beginning of JSON document"); + // The JSON root is either a struct or list + if (is_nested_token(tokens_gpu[offset])) { + std::cout << "Nested value at root. JSON doc with: " << num_tokens_out[0] << " tokens \n"; + std::cout << (tokens_gpu[offset] == token_t::StructBegin ? "[StructEnd]" : "[ListBegin]") + << "\n"; + // Initialize the root column and append this row to it + append_row_to_column(&root_column, + row_offset_zero, + token_to_column_type(tokens_gpu[offset]), + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + 0); + + // Push the root node onto the stack for the data path + current_data_path.push({&root_column, row_offset_zero, nullptr, zero_child_count}); + + // Continue with the next token from the token stream + offset++; + } + // The JSON is a simple scalar value -> create simple table and return + else { + std::cout << "Scalar value at root. JSON doc with: " << num_tokens_out[0] << " tokens \n"; + + constexpr SymbolOffsetT max_tokens_for_scalar_value = 2; + CUDF_EXPECTS(num_tokens_out[0] <= max_tokens_for_scalar_value, + "Invalid JSON format. Expected just a scalar value."); + + // If this isn't the only token, verify the subsequent token is the correct end-of-* partner + if ((offset + 1) < num_tokens_out[0]) { + CUDF_EXPECTS(tokens_gpu[offset + 1] == end_of_partner(tokens_gpu[offset]), + "Invalid JSON token sequence"); + } + + // The offset to the first symbol from the JSON input associated with the current token + auto const& token_begin_offset = get_token_index(tokens_gpu[offset], token_indices_gpu[offset]); + + // The offset to one past the last symbol associated with the current token + // Literals without trailing space are missing the corresponding end-of-* counterpart. + auto const& token_end_offset = + (offset + 1 < num_tokens_out[0]) + ? get_token_index(tokens_gpu[offset + 1], token_indices_gpu[offset + 1]) + : input.size(); + + root_column.type = json_col_t::StringColumn; + root_column.string_offsets.push_back(token_begin_offset); + root_column.string_lengths.push_back(token_end_offset - token_begin_offset); + return root_column; + } + + //-------------------------------------------------------------------------------- + // TRAVERSE JSON TREE + //-------------------------------------------------------------------------------- + auto get_target_row_index = [](std::stack<tree_node> const& current_data_path, + json_column* target_column) { + std::cout << " -> target row: " + << ((current_data_path.top().column->type == json_col_t::ListColumn) + ? target_column->current_offset + : current_data_path.top().row_index) + << "\n"; + return (current_data_path.top().column->type == json_col_t::ListColumn) + ? target_column->current_offset + : current_data_path.top().row_index; + }; + + while (offset < num_tokens_out[0]) { + // Verify there's at least the JSON root node left on the stack to which we can append data + CUDF_EXPECTS(current_data_path.size() > 0, "Invalid JSON structure"); + + // Verify that the current node in the tree (which becomes this nodes parent) can have children + CUDF_EXPECTS(current_data_path.top().column->type == json_col_t::ListColumn or + current_data_path.top().column->type == json_col_t::StructColumn, + "Invalid JSON structure"); + + // The token we're currently parsing + auto const& token = tokens_gpu[offset]; + + // StructBegin token + if (token == token_t::StructBegin) { + std::cout << "[StructBegin]\n"; + // Get this node's column. That is, the parent node's selected column: + // (a) if parent is a list, then this will (create and) return the list's only child column + // (b) if parent is a struct, then this will return the column selected by the last field name + // encountered. + json_column* selected_col = get_selected_column(current_data_path); + + // Get the row offset at which to insert + auto target_row_index = get_target_row_index(current_data_path, selected_col); + + // Increment parent's child count and insert this struct node into the data path + current_data_path.top().num_children++; + current_data_path.push({selected_col, target_row_index, nullptr, zero_child_count}); + + // Add this struct node to the current column + append_row_to_column(selected_col, + target_row_index, + token_to_column_type(tokens_gpu[offset]), + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + zero_child_count); + } + + // StructEnd token + else if (token == token_t::StructEnd) { + std::cout << "[StructEnd]\n"; + // Verify that this node in fact a struct node (i.e., it was part of a struct column) + CUDF_EXPECTS(current_data_path.top().column->type == json_col_t::StructColumn, + "Broken invariant while parsing JSON"); + CUDF_EXPECTS(current_data_path.top().column != nullptr, + "Broken invariant while parsing JSON"); + + // Update row to account for string offset + update_row(current_data_path.top().column, + current_data_path.top().row_index, + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + current_data_path.top().num_children); + + // Pop struct from the path stack + current_data_path.pop(); + } + + // ListBegin token + else if (token == token_t::ListBegin) { + std::cout << "[ListBegin]\n"; + // Get the selected column + json_column* selected_col = get_selected_column(current_data_path); + + // Get the row offset at which to insert + auto target_row_index = get_target_row_index(current_data_path, selected_col); + + // Increment parent's child count and insert this struct node into the data path + current_data_path.top().num_children++; + current_data_path.push({selected_col, target_row_index, nullptr, zero_child_count}); + + // Add this struct node to the current column + append_row_to_column(selected_col, + target_row_index, + token_to_column_type(tokens_gpu[offset]), + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + zero_child_count); + } + + // ListEnd token + else if (token == token_t::ListEnd) { + std::cout << "[ListEnd]\n"; + // Verify that this node in fact a list node (i.e., it was part of a list column) + CUDF_EXPECTS(current_data_path.top().column->type == json_col_t::ListColumn, + "Broken invariant while parsing JSON"); + CUDF_EXPECTS(current_data_path.top().column != nullptr, + "Broken invariant while parsing JSON"); + + // Update row to account for string offset + update_row(current_data_path.top().column, + current_data_path.top().row_index, + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]), + current_data_path.top().num_children); + + // Pop list from the path stack + current_data_path.pop(); + } + + // Error token + else if (token == token_t::ErrorBegin) { + std::cout << "[ErrorBegin]\n"; + CUDF_FAIL("Parser encountered an invalid format."); + } + + // FieldName, String, or Value (begin, end)-pair + else if (token == token_t::FieldNameBegin or token == token_t::StringBegin or + token == token_t::ValueBegin) { + // Verify that this token has the right successor to build a correct (being, end) token pair + CUDF_EXPECTS((offset + 1) < num_tokens_out[0], "Invalid JSON token sequence"); + CUDF_EXPECTS(tokens_gpu[offset + 1] == end_of_partner(token), "Invalid JSON token sequence"); + + // The offset to the first symbol from the JSON input associated with the current token + auto const& token_begin_offset = + get_token_index(tokens_gpu[offset], token_indices_gpu[offset]); + + // The offset to one past the last symbol associated with the current token + auto const& token_end_offset = + get_token_index(tokens_gpu[offset + 1], token_indices_gpu[offset + 1]); + + // FieldNameBegin + // For the current struct node in the tree, select the child column corresponding to this + // field name + if (token == token_t::FieldNameBegin) { + std::cout << "[FieldNameBegin]\n"; + std::string field_name{input.data() + token_begin_offset, + (token_end_offset - token_begin_offset)}; + current_data_path.top().current_selected_col = select_column(current_data_path, field_name); + } + // StringBegin + // ValueBegin + // As we currently parse to string columns there's no further differentiation + else if (token == token_t::StringBegin or token == token_t::ValueBegin) { + std::cout << "[StringBegin/ValueBegin]\n"; + // Get the selected column + json_column* selected_col = get_selected_column(current_data_path); + + // Get the row offset at which to insert + auto target_row_index = get_target_row_index(current_data_path, selected_col); + + current_data_path.top().num_children++; + + append_row_to_column(selected_col, + target_row_index, + token_to_column_type(token), + token_begin_offset, + token_end_offset, + zero_child_count); + } else { + CUDF_FAIL("Unknown JSON token"); + } + + // As we've also consumed the end-of-* token, we advance the processed token offset by one + offset++; + } + + offset++; + } + + // Make sure all of a struct's child columns have the same length + + + return root_column; +} + +} // namespace detail } // namespace cudf::io::json diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 7d74a33adc4..82193046664 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -23,8 +23,103 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> +#include <stack> +#include <string> + namespace nested_json = cudf::io::json; +namespace { +void print_column(std::string const& input, + nested_json::json_column const& column, + uint32_t indent = 0); +void print_json_string_col(std::string const& input, + nested_json::json_column const& column, + uint32_t indent = 0) +{ + for (std::size_t i = 0; i < column.string_offsets.size(); i++) { + std::cout << i << ": [" << (column.validity[i]?"1":"0") << "] '" << input.substr(column.string_offsets[i], column.string_lengths[i]) + << "'\n"; + } +} + +std::string pad(uint32_t indent = 0) +{ + std::string pad{}; + if (indent > 0) pad.insert(pad.begin(), indent, ' '); + return pad; +} + +void print_json_list_col(std::string const& input, + nested_json::json_column const& column, + uint32_t indent = 0) +{ + std::cout << pad(indent) << " [LIST]\n"; + std::cout << pad(indent) << " -> num. child-columns: " << column.child_columns.size() << "\n"; + std::cout << pad(indent) << " -> num. rows: " << column.current_offset << "\n"; + std::cout << pad(indent) << " -> num. valid: " << column.valid_count << "\n"; + std::cout << pad(indent) << " offsets[]: " + << "\n"; + for (std::size_t i = 0; i < column.child_offsets.size() - 1; i++) { + std::cout << pad(indent + 2) << i << ": [" << (column.validity[i]?"1":"0") << "] [" << column.child_offsets[i] << ", " + << column.child_offsets[i + 1] << ")\n"; + } + if (column.child_columns.size() > 0) { + std::cout << pad(indent) << column.child_columns.begin()->first << "[]: " + << "\n"; + print_column(input, column.child_columns.begin()->second, indent + 2); + } +} + +void print_json_struct_col(std::string const& input, + nested_json::json_column const& column, + uint32_t indent = 0) +{ + std::cout << pad(indent) << " [STRUCT]\n"; + std::cout << pad(indent) << " -> num. child-columns: " << column.child_columns.size() << "\n"; + std::cout << pad(indent) << " -> num. rows: " << column.current_offset << "\n"; + std::cout << pad(indent) << " -> num. valid: " << column.valid_count << "\n"; + std::cout << pad(indent) << " -> validity[]: " + << "\n"; + for (std::size_t i = 0; i < column.current_offset; i++) { + std::cout << pad(indent + 2) << i << ": [" << (column.validity[i]?"1":"0") << "]\n"; + } + auto it = std::begin(column.child_columns); + for (std::size_t i = 0; i < column.child_columns.size(); i++) { + std::cout << "child #" << i << " '" << it->first << "'[] \n"; + print_column(input, it->second, indent + 2); + it++; + } +} + +void print_column(std::string const& input, nested_json::json_column const& column, uint32_t indent) +{ + switch (column.type) { + case nested_json::json_col_t::StringColumn: print_json_string_col(input, column, indent); break; + case nested_json::json_col_t::ListColumn: print_json_list_col(input, column, indent); break; + case nested_json::json_col_t::StructColumn: print_json_struct_col(input, column, indent); break; + case nested_json::json_col_t::Unknown: std::cout << pad(indent) << "[UNKNOWN]\n"; break; + default: break; + } +} + +void print_json_cols(std::string const& input, nested_json::json_column const& column) +{ + auto column_type_string = [](nested_json::json_col_t column_type) { + switch (column_type) { + case nested_json::json_col_t::Unknown: return "Unknown"; + case nested_json::json_col_t::ListColumn: return "List"; + case nested_json::json_col_t::StructColumn: return "Struct"; + case nested_json::json_col_t::StringColumn: return "String"; + default: return "Unknown"; + } + }; + + std::cout << "TYPE: " << column_type_string(column.type) << "\n"; + print_column(input, column, 0); +}; + +} // namespace + // Base test fixture for tests struct JsonTest : public cudf::test::BaseFixture { }; @@ -189,3 +284,34 @@ TEST_F(JsonTest, TokenStream) EXPECT_EQ(golden_token_stream[i].second, tokens_gpu[i]) << "Mismatch at #" << i; } } + +TEST_F(JsonTest, Simple) +{ + using nested_json::PdaTokenT; + using nested_json::SymbolOffsetT; + using nested_json::SymbolT; + + // Prepare cuda stream for data transfers & kernels + cudaStream_t stream = nullptr; + cudaStreamCreate(&stream); + rmm::cuda_stream_view stream_view(stream); + + // + // std::string input = R"( ["foo", null, "bar"] )"; + std::string input = R"( [{"a":0.0, "b":0.1}, {"b":1.1}] )"; + // std::string input = R"( [[1], [2], null, [3], [4]] )"; + + // String / value + // std::string input = R"( " Foobar" )"; + // std::string input = R"( 123.456 )"; + // std::string input = R"( 123.456)"; + // std::string input = R"(null)"; + // std::string input = R"( [null, [2], null, [3], [4]] )"; // <= will fail because col will be + // inferred as string/val column + + // Get the JSON's tree representation + auto json_root_col = nested_json::detail::get_json_columns( + cudf::host_span<SymbolT const>{input.data(), input.size()}, stream_view); + + print_json_cols(input, json_root_col); +} From f015594d063a9ed84538be821d0843bd942d3de0 Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Wed, 27 Jul 2022 23:51:14 +0530 Subject: [PATCH 63/81] add parse_json_to_columns -> cudf::column --- cpp/src/io/json/nested_json.hpp | 10 +++ cpp/src/io/json/nested_json_gpu.cu | 132 ++++++++++++++++++++++++++--- cpp/tests/io/nested_json_test.cu | 47 ++++++++-- 3 files changed, 171 insertions(+), 18 deletions(-) diff --git a/cpp/src/io/json/nested_json.hpp b/cpp/src/io/json/nested_json.hpp index e0cd2de9286..7cb16b52465 100644 --- a/cpp/src/io/json/nested_json.hpp +++ b/cpp/src/io/json/nested_json.hpp @@ -204,6 +204,16 @@ void get_token_stream(device_span<SymbolT const> d_json_in, */ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream); +/** + * @brief Parses the given JSON string and generates a cudf::column of the given input. + * + * @param input The JSON input + * @param stream The CUDA stream to which kernels are dispatched + * @return cudf::column of the given input + */ +std::unique_ptr<column> parse_json_to_columns(host_span<SymbolT const> input, + rmm::cuda_stream_view stream); + } // namespace detail } // namespace cudf::io::json diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 131522c8c66..06bd2d0a46e 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -21,12 +21,17 @@ #include <io/fst/lookup_tables.cuh> #include <io/utilities/hostdevice_vector.hpp> +#include <cudf/column/column_factories.hpp> #include <cudf/types.hpp> #include <cudf/utilities/span.hpp> #include <iterator> +#include <rmm/device_uvector.hpp> #include <rmm/exec_policy.hpp> +#include <thrust/device_vector.h> +#include <thrust/transform.h> + #include <stack> namespace cudf::io::json { @@ -653,6 +658,104 @@ struct tree_node { std::size_t num_children = 0; }; +std::pair<std::unique_ptr<column>, std::vector<std::string>> json_column_to_cudf_column( + json_column const& json_col, device_span<SymbolT const> d_input, rmm::cuda_stream_view stream) +{ + std::unique_ptr<column> cudf_col; + switch (json_col.type) { + case json_col_t::StringColumn: { + // move string_offsets to GPU. + // transform string_offsets + d_input to string_data_ptrs. + // transform string_lengths to string_data_lengths. + auto col_size = json_col.string_offsets.size(); + using char_length_pair_t = thrust::pair<const char*, size_type>; + CUDF_EXPECTS(json_col.string_offsets.size() == json_col.string_lengths.size(), + "string offset, string length mismatch"); + rmm::device_uvector<char_length_pair_t> d_string_data(col_size, stream); + thrust::device_vector<json_column::row_offset_t> d_string_offsets( + json_col.string_offsets); // TODO device_uvector. + thrust::device_vector<json_column::row_offset_t> d_string_lengths( + json_col.string_lengths); // TODO device_uvector. + // TODO: what do we do with the column names? + auto offset_length_it = + thrust::make_zip_iterator(d_string_offsets.begin(), d_string_lengths.begin()); + thrust::transform(rmm::exec_policy(stream), + offset_length_it, + offset_length_it + col_size, + d_string_data.data(), + [data = d_input.data()] __device__(auto ip) { + return char_length_pair_t{data + thrust::get<0>(ip), thrust::get<1>(ip)}; + }); + return {make_strings_column(d_string_data, stream), {}}; + break; + } + case json_col_t::StructColumn: { + // Create empty struct column + std::vector<std::unique_ptr<column>> child_columns; + std::vector<std::string> column_names{}; + size_type num_rows{-1}; + // Create children columns + for (auto const& col : json_col.child_columns) { + column_names.push_back(col.first); + auto const& child_col = col.second; + auto [child_column, name] = json_column_to_cudf_column(child_col, d_input, stream); + num_rows = child_column->size(); + // TODO add to struct column + child_columns.push_back(std::move(child_column)); + } + return {make_structs_column(num_rows, + std::move(child_columns), + cudf::UNKNOWN_NULL_COUNT, + rmm::device_buffer{}, + stream), // TODO mr. + column_names}; + break; + } + case json_col_t::ListColumn: { + size_type num_rows = json_col.child_offsets.size(); + std::vector<std::string> cn{}; + + rmm::device_uvector<json_column::row_offset_t> d_offsets(num_rows, stream); + cudaMemcpyAsync(d_offsets.data(), + json_col.child_offsets.data(), + num_rows * sizeof(json_col.child_offsets[0]), + cudaMemcpyHostToDevice, + stream); + auto offsets_column = + std::make_unique<column>(data_type{type_id::INT32}, num_rows, d_offsets.release()); + // Create children column + auto [child_column, name] = + json_column_to_cudf_column(json_col.child_columns.begin()->second, d_input, stream); + return {make_lists_column(num_rows - 1, + std::move(offsets_column), + std::move(child_column), + cudf::UNKNOWN_NULL_COUNT, + rmm::device_buffer{}, + stream), + std::move(cn)}; + break; + } + default: CUDF_FAIL("Unsupported column type, yet to be implemented"); break; + } + std::vector<std::string> cn{}; + return {std::move(cudf_col), std::move(cn)}; +} +// TODO cleanup. +json_column get_json_columns2(host_span<SymbolT const> input, + device_span<SymbolT const> d_input, + rmm::cuda_stream_view stream); +std::unique_ptr<column> parse_json_to_columns(host_span<SymbolT const> input, + rmm::cuda_stream_view stream) +{ + // Allocate device memory for the JSON input & copy over to device + rmm::device_uvector<SymbolT> d_input{input.size(), stream}; + cudaMemcpyAsync( + d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); + auto root_column = get_json_columns2(input, {d_input.data(), d_input.size()}, stream); + return std::move( + json_column_to_cudf_column(root_column, {d_input.data(), d_input.size()}, stream).first); +} + /** * RULES: * @@ -662,6 +765,17 @@ struct tree_node { * --> else (e.g., 'null', '123', '"foo"'): it becomes a string column */ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) +{ + // Allocate device memory for the JSON input & copy over to device + rmm::device_uvector<SymbolT> d_input{input.size(), stream}; + cudaMemcpyAsync( + d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); + return get_json_columns2(input, {d_input.data(), d_input.size()}, stream); +} + +json_column get_json_columns2(host_span<SymbolT const> input, + device_span<SymbolT const> d_input, + rmm::cuda_stream_view stream) { // Default name for a list's child column std::string const list_child_name = "items"; @@ -671,13 +785,8 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi hostdevice_vector<SymbolOffsetT> token_indices_gpu{input.size(), stream}; hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream}; - // Allocate device memory for the JSON input & copy over to device - rmm::device_uvector<SymbolT> d_input{input.size(), stream}; - cudaMemcpyAsync( - d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); - // Parse the JSON and get the token stream - get_token_stream(cudf::device_span<SymbolT>{d_input.data(), d_input.size()}, + get_token_stream(d_input, tokens_gpu.device_ptr(), token_indices_gpu.device_ptr(), num_tokens_out.device_ptr(), @@ -758,8 +867,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi uint32_t string_offset, uint32_t string_end, uint32_t child_count) { - - CUDF_EXPECTS(column != nullptr, "Encountered invalid JSON token sequence"); std::cout << " -> append_row_to_column()\n"; @@ -779,12 +886,12 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // Check for the last inserted row offset auto populated_row_count = column->current_offset; if (populated_row_count < row_index) { - std::cout << " ---> filling [" << populated_row_count << ", " << row_index << ")\n"; } // Fill all the omitted rows with "empty"/null rows - std::fill_n(std::back_inserter(column->validity), row_index - column->string_offsets.size(), false); + std::fill_n( + std::back_inserter(column->validity), row_index - column->string_offsets.size(), false); std::fill_n(std::back_inserter(column->string_offsets), row_index - column->string_offsets.size(), (column->string_offsets.size() > 0) ? column->string_offsets.back() : 0); @@ -904,8 +1011,8 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi json_column root_column; // Giving names to magic constants - constexpr uint32_t row_offset_zero = 0; - constexpr uint32_t zero_child_count = 0; + constexpr uint32_t row_offset_zero = 0; + constexpr uint32_t zero_child_count = 0; //-------------------------------------------------------------------------------- // INITIALIZE JSON ROOT NODE @@ -1137,7 +1244,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // Make sure all of a struct's child columns have the same length - return root_column; } diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 82193046664..aeb9bf3765b 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -14,12 +14,16 @@ * limitations under the License. */ +#include "cudf_test/column_wrapper.hpp" #include <io/json/nested_json.hpp> #include <io/utilities/hostdevice_vector.hpp> #include <cudf_test/base_fixture.hpp> +#include <cudf_test/column_utilities.hpp> #include <cudf_test/cudf_gtest.hpp> +#include <cudf/lists/lists_column_view.hpp> + #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> @@ -37,8 +41,8 @@ void print_json_string_col(std::string const& input, uint32_t indent = 0) { for (std::size_t i = 0; i < column.string_offsets.size(); i++) { - std::cout << i << ": [" << (column.validity[i]?"1":"0") << "] '" << input.substr(column.string_offsets[i], column.string_lengths[i]) - << "'\n"; + std::cout << i << ": [" << (column.validity[i] ? "1" : "0") << "] '" + << input.substr(column.string_offsets[i], column.string_lengths[i]) << "'\n"; } } @@ -60,8 +64,8 @@ void print_json_list_col(std::string const& input, std::cout << pad(indent) << " offsets[]: " << "\n"; for (std::size_t i = 0; i < column.child_offsets.size() - 1; i++) { - std::cout << pad(indent + 2) << i << ": [" << (column.validity[i]?"1":"0") << "] [" << column.child_offsets[i] << ", " - << column.child_offsets[i + 1] << ")\n"; + std::cout << pad(indent + 2) << i << ": [" << (column.validity[i] ? "1" : "0") << "] [" + << column.child_offsets[i] << ", " << column.child_offsets[i + 1] << ")\n"; } if (column.child_columns.size() > 0) { std::cout << pad(indent) << column.child_columns.begin()->first << "[]: " @@ -81,7 +85,7 @@ void print_json_struct_col(std::string const& input, std::cout << pad(indent) << " -> validity[]: " << "\n"; for (std::size_t i = 0; i < column.current_offset; i++) { - std::cout << pad(indent + 2) << i << ": [" << (column.validity[i]?"1":"0") << "]\n"; + std::cout << pad(indent + 2) << i << ": [" << (column.validity[i] ? "1" : "0") << "]\n"; } auto it = std::begin(column.child_columns); for (std::size_t i = 0; i < column.child_columns.size(); i++) { @@ -315,3 +319,36 @@ TEST_F(JsonTest, Simple) print_json_cols(input, json_root_col); } + +TEST_F(JsonTest, ExtractColumn) +{ + using nested_json::PdaTokenT; + using nested_json::SymbolOffsetT; + using nested_json::SymbolT; + + // Prepare cuda stream for data transfers & kernels + cudaStream_t stream = nullptr; + cudaStreamCreate(&stream); + rmm::cuda_stream_view stream_view(stream); + + std::string input = R"( [{"a":0.0, "b":1.0}, {"a":0.1, "b":1.1}, {"a":0.2, "b":1.2}] )"; + // Get the JSON's tree representation + auto cudf_column = nested_json::detail::parse_json_to_columns( + cudf::host_span<SymbolT const>{input.data(), input.size()}, stream_view); + + std::cout << std::endl << "=== PARSED COLUMN ===" << std::endl; + cudf::test::print(*cudf_column); + cudf::column_view cudf_struct_view = + cudf_column->child(cudf::lists_column_view::child_column_index); + + auto expected_col1 = cudf::test::strings_column_wrapper({"0.0", "0.1", "0.2"}); + auto expected_col2 = cudf::test::strings_column_wrapper({"1.0", "1.1", "1.2"}); + cudf::column_view parsed_col1 = cudf_struct_view.child(0); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected_col1, parsed_col1); + std::cout << "*parsed_col1:\n"; + cudf::test::print(parsed_col1); + cudf::column_view parsed_col2 = cudf_struct_view.child(1); + CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected_col2, parsed_col2); + std::cout << "*parsed_col2:\n"; + cudf::test::print(parsed_col2); +} From 389e8e8619b3f84da6743576eeef9b7013961aa8 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 27 Jul 2022 15:57:25 -0700 Subject: [PATCH 64/81] + wraps dbg print in macro + fills/levels child columns --- cpp/src/io/json/nested_json.hpp | 46 +++++++ cpp/src/io/json/nested_json_gpu.cu | 186 +++++++++++++++-------------- cpp/tests/io/nested_json_test.cu | 66 +++++----- 3 files changed, 179 insertions(+), 119 deletions(-) diff --git a/cpp/src/io/json/nested_json.hpp b/cpp/src/io/json/nested_json.hpp index e0cd2de9286..f1b85edf250 100644 --- a/cpp/src/io/json/nested_json.hpp +++ b/cpp/src/io/json/nested_json.hpp @@ -128,6 +128,52 @@ struct json_column { // Counting the current number of items in this column row_offset_t current_offset = 0; + + /** + * @brief Fills the rows up to the given \p up_to_row_offset with nulls. + * + * @param up_to_row_offset The row offset up to which to fill with nulls. + */ + void null_fill(row_offset_t up_to_row_offset) + { + // Fill all the rows up to up_to_row_offset with "empty"/null rows + std::fill_n(std::back_inserter(validity), up_to_row_offset - string_offsets.size(), false); + std::fill_n(std::back_inserter(string_offsets), + up_to_row_offset - string_offsets.size(), + (string_offsets.size() > 0) ? string_offsets.back() : 0); + std::fill_n(std::back_inserter(string_lengths), up_to_row_offset - string_lengths.size(), 0); + std::fill_n(std::back_inserter(child_offsets), + up_to_row_offset + 1 - child_offsets.size(), + (child_offsets.size() > 0) ? child_offsets.back() : 0); + current_offset = up_to_row_offset; + } + + /** + * @brief Recursively iterates through the tree of columns making sure that all child columns of a + * struct column have the same row count, filling missing rows with nulls. + * + * @param min_row_count The minimum number of rows to be filled. + */ + void level_child_cols_recursively(row_offset_t min_row_count) + { + // Fill this columns with nulls up to the given row count + null_fill(min_row_count); + + // If this is a struct column, we need to level all its child columns + if (type == json_col_t::StructColumn) { + for (auto it = std::begin(child_columns); it != std::end(child_columns); it++) { + it->second.level_child_cols_recursively(min_row_count); + } + } + // If this is a list column, we need to make sure that its child column levels its children + else if (type == json_col_t::ListColumn) { + auto it = std::begin(child_columns); + // Make that child column fill its child columns up to its own row count + if (it != std::end(child_columns)) { + it->second.level_child_cols_recursively(it->second.current_offset); + } + } + } }; /** diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 131522c8c66..0baae3c8b4c 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -29,6 +29,11 @@ #include <stack> +// Debug print flag +#ifndef NJP_DEBUG_PRINT +#define NJP_DEBUG_PRINT +#endif + namespace cudf::io::json { //------------------------------------------------------------------------------ @@ -637,9 +642,12 @@ void get_token_stream(device_span<SymbolT const> d_json_in, stream); } +/** + * @brief A tree node contains all the information of the data + * + */ struct tree_node { // The column that this node is associated with - // E.g., if this is a struct node, it's the struct's column json_column* column; // The row offset that this node belongs to within the given column @@ -653,14 +661,6 @@ struct tree_node { std::size_t num_children = 0; }; -/** - * RULES: - * - * if first node type of a column is: - * --> a list: it becomes list column - * --> a struct: it becomes struct column - * --> else (e.g., 'null', '123', '"foo"'): it becomes a string column - */ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) { // Default name for a list's child column @@ -742,6 +742,7 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi }; }; +#ifdef NJP_DEBUG_PRINT auto column_type_string = [](json_col_t column_type) { switch (column_type) { case json_col_t::Unknown: return "Unknown"; @@ -752,16 +753,33 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi } }; - auto append_row_to_column = [&input, &column_type_string](json_column* column, - uint32_t row_index, - json_col_t const& row_type, - uint32_t string_offset, - uint32_t string_end, - uint32_t child_count) { - - + auto token_to_string = [](PdaTokenT token_type) { + switch (token_type) { + case token_t::StructBegin: return "StructBegin"; + case token_t::StructEnd: return "StructEnd"; + case token_t::ListBegin: return "ListBegin"; + case token_t::ListEnd: return "ListEnd"; + case token_t::FieldNameBegin: return "FieldNameBegin"; + case token_t::FieldNameEnd: return "FieldNameEnd"; + case token_t::StringBegin: return "StringBegin"; + case token_t::StringEnd: return "StringEnd"; + case token_t::ValueBegin: return "ValueBegin"; + case token_t::ValueEnd: return "ValueEnd"; + case token_t::ErrorBegin: return "ErrorBegin"; + default: return "Unknown"; + } + }; +#endif + + auto append_row_to_column = [&](json_column* column, + uint32_t row_index, + json_col_t const& row_type, + uint32_t string_offset, + uint32_t string_end, + uint32_t child_count) { CUDF_EXPECTS(column != nullptr, "Encountered invalid JSON token sequence"); +#ifdef NJP_DEBUG_PRINT std::cout << " -> append_row_to_column()\n"; std::cout << " ---> col@" << column << "\n"; std::cout << " ---> row #" << row_index << "\n"; @@ -769,33 +787,20 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi std::cout << " ---> row_type: " << column_type_string(row_type) << "\n"; std::cout << " ---> string: '" << std::string{input.data() + string_offset, (string_end - string_offset)} << "'\n"; +#endif // If, thus far, the column's type couldn't be inferred, we infer it to the given type if (column->type == json_col_t::Unknown) { column->type = row_type; } - // We shouldn't run into this, as we shouldn't + // We shouldn't run into this, as we shouldn't be asked to append an "unknown" row type CUDF_EXPECTS(column->type != json_col_t::Unknown, "Encountered invalid JSON token sequence"); - // Check for the last inserted row offset - auto populated_row_count = column->current_offset; - if (populated_row_count < row_index) { + // Fill all the omitted rows with "empty"/null rows (if needed) + column->null_fill(row_index); - std::cout << " ---> filling [" << populated_row_count << ", " << row_index << ")\n"; - } - - // Fill all the omitted rows with "empty"/null rows - std::fill_n(std::back_inserter(column->validity), row_index - column->string_offsets.size(), false); - std::fill_n(std::back_inserter(column->string_offsets), - row_index - column->string_offsets.size(), - (column->string_offsets.size() > 0) ? column->string_offsets.back() : 0); - std::fill_n( - std::back_inserter(column->string_lengths), row_index - column->string_lengths.size(), 0); - std::fill_n(std::back_inserter(column->child_offsets), - row_index + 1 - column->child_offsets.size(), - (column->child_offsets.size() > 0) ? column->child_offsets.back() : 0); - - // Fill all the omitted rows with "empty" rows - // col type | row type => + // Table listing what we intend to use for a given column type and row type combination + // col type | row type => {valid, FAIL, null} + // ----------------------------------------------- // List | List => valid // List | Struct => FAIL // List | String => null @@ -815,8 +820,13 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi column->current_offset++; }; + /** + * @brief Updates the given row in the given column with a new string_end and child_count. In + * particular, updating the child count is relevant for list columns. + */ auto update_row = [](json_column* column, uint32_t row_index, uint32_t string_end, uint32_t child_count) { +#ifdef NJP_DEBUG_PRINT std::cout << " -> update_row()\n"; std::cout << " ---> col@" << column << "\n"; std::cout << " ---> row #" << row_index << "\n"; @@ -824,6 +834,7 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi << "\n"; std::cout << " ---> child_offsets = " << (column->child_offsets[row_index + 1] + child_count) << "\n"; +#endif column->string_lengths[row_index] = column->child_offsets[row_index + 1] + child_count; column->child_offsets[row_index + 1] = column->child_offsets[row_index + 1] + child_count; }; @@ -833,13 +844,11 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi * * That is, if \p current_data_path top-of-stack is * (a) a struct, the selected child column corresponds to the child column of the last field name - * node encoutnered. + * node encountered. * (b) a list, the selected child column corresponds to single child column of * the list column. In this case, the child column may not exist yet. - * */ auto get_selected_column = [&list_child_name](std::stack<tree_node>& current_data_path) { - std::cout << " -> get_selected_column()\n"; json_column* selected_col = current_data_path.top().current_selected_col; // If the node does not have a selected column yet @@ -855,19 +864,26 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi } current_data_path.top().current_selected_col = ¤t_data_path.top().column->child_columns.begin()->second; - std::cout << " ---> selected col@" << current_data_path.top().current_selected_col << "\n"; - return current_data_path.top().current_selected_col; + selected_col = current_data_path.top().current_selected_col; } else { CUDF_FAIL("Trying to retrieve child column without encountering a field name."); } - } else { - std::cout << " ---> selected col@" << selected_col << "\n"; - return selected_col; } +#ifdef NJP_DEBUG_PRINT + std::cout << " -> get_selected_column()\n"; + std::cout << " ---> selected col@" << selected_col << "\n"; +#endif + return selected_col; }; + /** + * @brief Returns a pointer to the child column with the given \p field_name within the current + * struct column. + */ auto select_column = [](std::stack<tree_node>& current_data_path, std::string const& field_name) { +#ifdef NJP_DEBUG_PRINT std::cout << " -> select_column(" << field_name << ")\n"; +#endif // The field name's parent struct node auto& current_struct_node = current_data_path.top(); @@ -880,20 +896,32 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // The field name's column exists already, select that as the struct node's currently selected // child column - if (child_col_it != struct_col->child_columns.end()) { - std::cout << " ---> selected col@" << &child_col_it->second << "\n"; - return &child_col_it->second; - } + if (child_col_it != struct_col->child_columns.end()) { return &child_col_it->second; } // The field name's column does not exist yet, so we have to append the child column to the // struct column - json_column* ptr = &struct_col->child_columns.insert({field_name, {}}).first->second; - json_column* ptr2 = &struct_col->child_columns.find(field_name)->second; - std::cout << "ptr1: " << static_cast<void*>(ptr) << ", ptr2: " << static_cast<void*>(ptr2) - << "\n"; return &struct_col->child_columns.insert({field_name, {}}).first->second; }; + /** + * @brief Gets the row offset at which to insert. I.e., for a child column of a list column, we + * just have to append the row to the end. Otherwise we have to propagate the row offset from the + * parent struct column. + */ + auto get_target_row_index = [](std::stack<tree_node> const& current_data_path, + json_column* target_column) { +#ifdef NJP_DEBUG_PRINT + std::cout << " -> target row: " + << ((current_data_path.top().column->type == json_col_t::ListColumn) + ? target_column->current_offset + : current_data_path.top().row_index) + << "\n"; +#endif + return (current_data_path.top().column->type == json_col_t::ListColumn) + ? target_column->current_offset + : current_data_path.top().row_index; + }; + // The stack represents the path from the JSON root node to the current node being looked at std::stack<tree_node> current_data_path; @@ -901,24 +929,21 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi std::size_t offset = 0; // The root column - json_column root_column; + json_column root_column{}; // Giving names to magic constants - constexpr uint32_t row_offset_zero = 0; - constexpr uint32_t zero_child_count = 0; + constexpr uint32_t row_offset_zero = 0; + constexpr uint32_t zero_child_count = 0; //-------------------------------------------------------------------------------- // INITIALIZE JSON ROOT NODE //-------------------------------------------------------------------------------- - // The JSON root may only a struct, list, string, or value node + // The JSON root may only be a struct, list, string, or value node CUDF_EXPECTS(num_tokens_out[0] > 0, "Empty JSON input not supported"); CUDF_EXPECTS(is_valid_root_token(tokens_gpu[offset]), "Invalid beginning of JSON document"); // The JSON root is either a struct or list if (is_nested_token(tokens_gpu[offset])) { - std::cout << "Nested value at root. JSON doc with: " << num_tokens_out[0] << " tokens \n"; - std::cout << (tokens_gpu[offset] == token_t::StructBegin ? "[StructEnd]" : "[ListBegin]") - << "\n"; // Initialize the root column and append this row to it append_row_to_column(&root_column, row_offset_zero, @@ -935,8 +960,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi } // The JSON is a simple scalar value -> create simple table and return else { - std::cout << "Scalar value at root. JSON doc with: " << num_tokens_out[0] << " tokens \n"; - constexpr SymbolOffsetT max_tokens_for_scalar_value = 2; CUDF_EXPECTS(num_tokens_out[0] <= max_tokens_for_scalar_value, "Invalid JSON format. Expected just a scalar value."); @@ -957,27 +980,15 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi ? get_token_index(tokens_gpu[offset + 1], token_indices_gpu[offset + 1]) : input.size(); - root_column.type = json_col_t::StringColumn; - root_column.string_offsets.push_back(token_begin_offset); - root_column.string_lengths.push_back(token_end_offset - token_begin_offset); + append_row_to_column(&root_column, + row_offset_zero, + json_col_t::StringColumn, + token_begin_offset, + token_end_offset, + zero_child_count); return root_column; } - //-------------------------------------------------------------------------------- - // TRAVERSE JSON TREE - //-------------------------------------------------------------------------------- - auto get_target_row_index = [](std::stack<tree_node> const& current_data_path, - json_column* target_column) { - std::cout << " -> target row: " - << ((current_data_path.top().column->type == json_col_t::ListColumn) - ? target_column->current_offset - : current_data_path.top().row_index) - << "\n"; - return (current_data_path.top().column->type == json_col_t::ListColumn) - ? target_column->current_offset - : current_data_path.top().row_index; - }; - while (offset < num_tokens_out[0]) { // Verify there's at least the JSON root node left on the stack to which we can append data CUDF_EXPECTS(current_data_path.size() > 0, "Invalid JSON structure"); @@ -990,9 +1001,12 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // The token we're currently parsing auto const& token = tokens_gpu[offset]; +#ifdef NJP_DEBUG_PRINT + std::cout << "[" << token_to_string(token) << "]\n"; +#endif + // StructBegin token if (token == token_t::StructBegin) { - std::cout << "[StructBegin]\n"; // Get this node's column. That is, the parent node's selected column: // (a) if parent is a list, then this will (create and) return the list's only child column // (b) if parent is a struct, then this will return the column selected by the last field name @@ -1017,7 +1031,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // StructEnd token else if (token == token_t::StructEnd) { - std::cout << "[StructEnd]\n"; // Verify that this node in fact a struct node (i.e., it was part of a struct column) CUDF_EXPECTS(current_data_path.top().column->type == json_col_t::StructColumn, "Broken invariant while parsing JSON"); @@ -1036,7 +1049,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // ListBegin token else if (token == token_t::ListBegin) { - std::cout << "[ListBegin]\n"; // Get the selected column json_column* selected_col = get_selected_column(current_data_path); @@ -1058,7 +1070,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // ListEnd token else if (token == token_t::ListEnd) { - std::cout << "[ListEnd]\n"; // Verify that this node in fact a list node (i.e., it was part of a list column) CUDF_EXPECTS(current_data_path.top().column->type == json_col_t::ListColumn, "Broken invariant while parsing JSON"); @@ -1100,7 +1111,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // For the current struct node in the tree, select the child column corresponding to this // field name if (token == token_t::FieldNameBegin) { - std::cout << "[FieldNameBegin]\n"; std::string field_name{input.data() + token_begin_offset, (token_end_offset - token_begin_offset)}; current_data_path.top().current_selected_col = select_column(current_data_path, field_name); @@ -1109,7 +1119,6 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi // ValueBegin // As we currently parse to string columns there's no further differentiation else if (token == token_t::StringBegin or token == token_t::ValueBegin) { - std::cout << "[StringBegin/ValueBegin]\n"; // Get the selected column json_column* selected_col = get_selected_column(current_data_path); @@ -1136,10 +1145,13 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_vi } // Make sure all of a struct's child columns have the same length - + root_column.level_child_cols_recursively(root_column.current_offset); return root_column; } } // namespace detail } // namespace cudf::io::json + +// Debug print flag +#undef NJP_DEBUG_PRINT diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 82193046664..927330ab9a4 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -29,19 +29,15 @@ namespace nested_json = cudf::io::json; namespace { + +// Forward declaration void print_column(std::string const& input, nested_json::json_column const& column, uint32_t indent = 0); -void print_json_string_col(std::string const& input, - nested_json::json_column const& column, - uint32_t indent = 0) -{ - for (std::size_t i = 0; i < column.string_offsets.size(); i++) { - std::cout << i << ": [" << (column.validity[i]?"1":"0") << "] '" << input.substr(column.string_offsets[i], column.string_lengths[i]) - << "'\n"; - } -} +/** + * @brief Helper to generate indentation + */ std::string pad(uint32_t indent = 0) { std::string pad{}; @@ -49,6 +45,19 @@ std::string pad(uint32_t indent = 0) return pad; } +/** + * @brief Prints a string column. + */ +void print_json_string_col(std::string const& input, + nested_json::json_column const& column, + uint32_t indent = 0) +{ + for (std::size_t i = 0; i < column.string_offsets.size(); i++) { + std::cout << pad(indent) << i << ": [" << (column.validity[i] ? "1" : "0") << "] '" + << input.substr(column.string_offsets[i], column.string_lengths[i]) << "'\n"; + } +} + void print_json_list_col(std::string const& input, nested_json::json_column const& column, uint32_t indent = 0) @@ -60,8 +69,8 @@ void print_json_list_col(std::string const& input, std::cout << pad(indent) << " offsets[]: " << "\n"; for (std::size_t i = 0; i < column.child_offsets.size() - 1; i++) { - std::cout << pad(indent + 2) << i << ": [" << (column.validity[i]?"1":"0") << "] [" << column.child_offsets[i] << ", " - << column.child_offsets[i + 1] << ")\n"; + std::cout << pad(indent + 2) << i << ": [" << (column.validity[i] ? "1" : "0") << "] [" + << column.child_offsets[i] << ", " << column.child_offsets[i + 1] << ")\n"; } if (column.child_columns.size() > 0) { std::cout << pad(indent) << column.child_columns.begin()->first << "[]: " @@ -81,11 +90,11 @@ void print_json_struct_col(std::string const& input, std::cout << pad(indent) << " -> validity[]: " << "\n"; for (std::size_t i = 0; i < column.current_offset; i++) { - std::cout << pad(indent + 2) << i << ": [" << (column.validity[i]?"1":"0") << "]\n"; + std::cout << pad(indent + 2) << i << ": [" << (column.validity[i] ? "1" : "0") << "]\n"; } auto it = std::begin(column.child_columns); for (std::size_t i = 0; i < column.child_columns.size(); i++) { - std::cout << "child #" << i << " '" << it->first << "'[] \n"; + std::cout << pad(indent + 2) << "child #" << i << " '" << it->first << "'[] \n"; print_column(input, it->second, indent + 2); it++; } @@ -102,22 +111,6 @@ void print_column(std::string const& input, nested_json::json_column const& colu } } -void print_json_cols(std::string const& input, nested_json::json_column const& column) -{ - auto column_type_string = [](nested_json::json_col_t column_type) { - switch (column_type) { - case nested_json::json_col_t::Unknown: return "Unknown"; - case nested_json::json_col_t::ListColumn: return "List"; - case nested_json::json_col_t::StructColumn: return "Struct"; - case nested_json::json_col_t::StringColumn: return "String"; - default: return "Unknown"; - } - }; - - std::cout << "TYPE: " << column_type_string(column.type) << "\n"; - print_column(input, column, 0); -}; - } // namespace // Base test fixture for tests @@ -298,8 +291,16 @@ TEST_F(JsonTest, Simple) // // std::string input = R"( ["foo", null, "bar"] )"; - std::string input = R"( [{"a":0.0, "b":0.1}, {"b":1.1}] )"; - // std::string input = R"( [[1], [2], null, [3], [4]] )"; + // std::string input = R"( [{"a":0.0, "c":{"c0":"0.2.0"}}, {"b":1.1}] )"; + // std::string input = R"( [{"a":0.0}, {"b":1.1, "c":{"c0":"1.2.0"}}] )"; + std::string input = R"( [{"a":0.0}, {"b":1.1, "c":{"c0":[[1],null,[2]]}}] )"; + // std::string input = + // R"( [{ "col0":[{"field1": 1, "field2": 2 }, null, {"field1": 3, "field2": 4 }, {"field1": 5, + // "field2": 6 }], "col1":"foo" }] )"; + // std::string input = R"( [ {"col1": 1, "col2": 2 }, {"col1": 3, "col2": 4 }, {"col1": 5, + // "col2": 6 }] )"; std::string input = R"( [ {"col1": 1, "col2": 2 }, null, {"col1": 3, "col2": + // 4 }, + // {"col1": 5, "col2": 6 }] )"; std::string input = R"( [[1], [2], null, [3], [4]] )"; // String / value // std::string input = R"( " Foobar" )"; @@ -313,5 +314,6 @@ TEST_F(JsonTest, Simple) auto json_root_col = nested_json::detail::get_json_columns( cudf::host_span<SymbolT const>{input.data(), input.size()}, stream_view); - print_json_cols(input, json_root_col); + std::cout << input << "\n"; + print_column(input, json_root_col); } From 42f7c4ae0bac6643d58850f55366ac439c1f0b3b Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Wed, 27 Jul 2022 16:14:47 -0700 Subject: [PATCH 65/81] disables debug print by default --- cpp/src/io/json/nested_json_gpu.cu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 0baae3c8b4c..d4e1b81ba21 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -31,7 +31,7 @@ // Debug print flag #ifndef NJP_DEBUG_PRINT -#define NJP_DEBUG_PRINT +//#define NJP_DEBUG_PRINT #endif namespace cudf::io::json { From ecf68fc57671eceaeee41354544f8492075a6f79 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 28 Jul 2022 04:13:31 -0700 Subject: [PATCH 66/81] changeing interface of get_json_columns to also take device_span --- cpp/src/io/json/nested_json.hpp | 7 ++- cpp/src/io/json/nested_json_gpu.cu | 71 ++++++++++++------------------ cpp/tests/io/nested_json_test.cu | 29 ++++++++---- 3 files changed, 53 insertions(+), 54 deletions(-) diff --git a/cpp/src/io/json/nested_json.hpp b/cpp/src/io/json/nested_json.hpp index 227321f619b..12e823efc0c 100644 --- a/cpp/src/io/json/nested_json.hpp +++ b/cpp/src/io/json/nested_json.hpp @@ -244,11 +244,14 @@ void get_token_stream(device_span<SymbolT const> d_json_in, /** * @brief Parses the given JSON string and generates a tree representation of the given input. * - * @param input The JSON input + * @param input The JSON input in host memory + * @param d_input The JSON input in device memory * @param stream The CUDA stream to which kernels are dispatched * @return */ -json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream); +json_column get_json_columns(host_span<SymbolT const> input, + device_span<SymbolT const> d_input, + rmm::cuda_stream_view stream); /** * @brief Parses the given JSON string and generates a cudf::column of the given input. diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index f63cc9d912b..36a7d043334 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -39,6 +39,28 @@ //#define NJP_DEBUG_PRINT #endif +namespace { + +/** + * @brief While parsing the token stream, we use a stack of tree_nodes to maintain all the + * information about the data path that is relevant. + */ +struct tree_node { + // The column that this node is associated with + cudf::io::json::json_column* column; + + // The row offset that this node belongs to within the given column + uint32_t row_index; + + // Selected child column + // E.g., if this is a struct node, and we subsequently encountered the field name "a", then this + // point's to the struct's "a" child column + cudf::io::json::json_column* current_selected_col = nullptr; + + std::size_t num_children = 0; +}; +} // namespace + namespace cudf::io::json { //------------------------------------------------------------------------------ @@ -647,25 +669,6 @@ void get_token_stream(device_span<SymbolT const> d_json_in, stream); } -/** - * @brief A tree node contains all the information of the data - * - */ -struct tree_node { - // The column that this node is associated with - json_column* column; - - // The row offset that this node belongs to within the given column - uint32_t row_index; - - // Selected child column - // E.g., if this is a struct node, and we subsequently encountered the field name "a", then this - // point's to the struct's "a" child column - json_column* current_selected_col = nullptr; - - std::size_t num_children = 0; -}; - std::pair<std::unique_ptr<column>, std::vector<std::string>> json_column_to_cudf_column( json_column const& json_col, device_span<SymbolT const> d_input, rmm::cuda_stream_view stream) { @@ -748,10 +751,7 @@ std::pair<std::unique_ptr<column>, std::vector<std::string>> json_column_to_cudf std::vector<std::string> cn{}; return {std::move(cudf_col), std::move(cn)}; } -// TODO cleanup. -json_column get_json_columns2(host_span<SymbolT const> input, - device_span<SymbolT const> d_input, - rmm::cuda_stream_view stream); + std::unique_ptr<column> parse_json_to_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) { @@ -759,31 +759,14 @@ std::unique_ptr<column> parse_json_to_columns(host_span<SymbolT const> input, rmm::device_uvector<SymbolT> d_input{input.size(), stream}; cudaMemcpyAsync( d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); - auto root_column = get_json_columns2(input, {d_input.data(), d_input.size()}, stream); + auto root_column = get_json_columns(input, {d_input.data(), d_input.size()}, stream); return std::move( json_column_to_cudf_column(root_column, {d_input.data(), d_input.size()}, stream).first); } -/** - * RULES: - * - * if first node type of a column is: - * --> a list: it becomes list column - * --> a struct: it becomes struct column - * --> else (e.g., 'null', '123', '"foo"'): it becomes a string column - */ -json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) -{ - // Allocate device memory for the JSON input & copy over to device - rmm::device_uvector<SymbolT> d_input{input.size(), stream}; - cudaMemcpyAsync( - d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); - return get_json_columns2(input, {d_input.data(), d_input.size()}, stream); -} - -json_column get_json_columns2(host_span<SymbolT const> input, - device_span<SymbolT const> d_input, - rmm::cuda_stream_view stream) +json_column get_json_columns(host_span<SymbolT const> input, + device_span<SymbolT const> d_input, + rmm::cuda_stream_view stream) { // Default name for a list's child column std::string const list_child_name = "items"; diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index bee15feeee7..bf3c2703063 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -15,6 +15,7 @@ */ #include "cudf_test/column_wrapper.hpp" + #include <io/json/nested_json.hpp> #include <io/utilities/hostdevice_vector.hpp> @@ -27,13 +28,11 @@ #include <rmm/cuda_stream.hpp> #include <rmm/cuda_stream_view.hpp> -#include <stack> #include <string> namespace nested_json = cudf::io::json; namespace { - // Forward declaration void print_column(std::string const& input, nested_json::json_column const& column, @@ -62,6 +61,9 @@ void print_json_string_col(std::string const& input, } } +/** + * @brief Prints a list column. + */ void print_json_list_col(std::string const& input, nested_json::json_column const& column, uint32_t indent = 0) @@ -83,6 +85,9 @@ void print_json_list_col(std::string const& input, } } +/** + * @brief Prints a struct column. + */ void print_json_struct_col(std::string const& input, nested_json::json_column const& column, uint32_t indent = 0) @@ -104,6 +109,9 @@ void print_json_struct_col(std::string const& input, } } +/** + * @brief Prints the column's data and recurses through and prints all the child columns. + */ void print_column(std::string const& input, nested_json::json_column const& column, uint32_t indent) { switch (column.type) { @@ -114,7 +122,6 @@ void print_column(std::string const& input, nested_json::json_column const& colu default: break; } } - } // namespace // Base test fixture for tests @@ -289,8 +296,7 @@ TEST_F(JsonTest, Simple) using nested_json::SymbolT; // Prepare cuda stream for data transfers & kernels - cudaStream_t stream = nullptr; - cudaStreamCreate(&stream); + rmm::cuda_stream stream{}; rmm::cuda_stream_view stream_view(stream); // @@ -314,9 +320,17 @@ TEST_F(JsonTest, Simple) // std::string input = R"( [null, [2], null, [3], [4]] )"; // <= will fail because col will be // inferred as string/val column + // Allocate device memory for the JSON input & copy over to device + rmm::device_uvector<SymbolT> d_input{input.size(), stream_view}; + cudaMemcpyAsync(d_input.data(), + input.data(), + input.size() * sizeof(input[0]), + cudaMemcpyHostToDevice, + stream.value()); + // Get the JSON's tree representation auto json_root_col = nested_json::detail::get_json_columns( - cudf::host_span<SymbolT const>{input.data(), input.size()}, stream_view); + cudf::host_span<SymbolT const>{input.data(), input.size()}, d_input, stream_view); std::cout << input << "\n"; print_column(input, json_root_col); @@ -329,8 +343,7 @@ TEST_F(JsonTest, ExtractColumn) using nested_json::SymbolT; // Prepare cuda stream for data transfers & kernels - cudaStream_t stream = nullptr; - cudaStreamCreate(&stream); + rmm::cuda_stream stream{}; rmm::cuda_stream_view stream_view(stream); std::string input = R"( [{"a":0.0, "b":1.0}, {"a":0.1, "b":1.1}, {"a":0.2, "b":1.2}] )"; From 93cbe1a6b200c6cfb710054f4e0af9dd5521a82a Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 28 Jul 2022 05:49:42 -0700 Subject: [PATCH 67/81] parsing to table_with_metadata --- cpp/src/io/json/nested_json.hpp | 4 +- cpp/src/io/json/nested_json_gpu.cu | 63 +++++++++++++++++++++++------- cpp/tests/io/nested_json_test.cu | 18 ++++++--- 3 files changed, 62 insertions(+), 23 deletions(-) diff --git a/cpp/src/io/json/nested_json.hpp b/cpp/src/io/json/nested_json.hpp index 12e823efc0c..75987215e7e 100644 --- a/cpp/src/io/json/nested_json.hpp +++ b/cpp/src/io/json/nested_json.hpp @@ -260,8 +260,8 @@ json_column get_json_columns(host_span<SymbolT const> input, * @param stream The CUDA stream to which kernels are dispatched * @return cudf::column of the given input */ -std::unique_ptr<column> parse_json_to_columns(host_span<SymbolT const> input, - rmm::cuda_stream_view stream); +table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, + rmm::cuda_stream_view stream); } // namespace detail diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 36a7d043334..cec8c3148e7 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -22,6 +22,7 @@ #include <io/utilities/hostdevice_vector.hpp> #include <cudf/column/column_factories.hpp> +#include <cudf/table/table.hpp> #include <cudf/types.hpp> #include <cudf/utilities/span.hpp> @@ -669,7 +670,7 @@ void get_token_stream(device_span<SymbolT const> d_json_in, stream); } -std::pair<std::unique_ptr<column>, std::vector<std::string>> json_column_to_cudf_column( +std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to_cudf_column( json_column const& json_col, device_span<SymbolT const> d_input, rmm::cuda_stream_view stream) { std::unique_ptr<column> cudf_col; @@ -703,16 +704,17 @@ std::pair<std::unique_ptr<column>, std::vector<std::string>> json_column_to_cudf case json_col_t::StructColumn: { // Create empty struct column std::vector<std::unique_ptr<column>> child_columns; - std::vector<std::string> column_names{}; + std::vector<column_name_info> column_names{}; size_type num_rows{-1}; // Create children columns for (auto const& col : json_col.child_columns) { - column_names.push_back(col.first); - auto const& child_col = col.second; - auto [child_column, name] = json_column_to_cudf_column(child_col, d_input, stream); - num_rows = child_column->size(); + column_names.emplace_back(col.first); + auto const& child_col = col.second; + auto [child_column, names] = json_column_to_cudf_column(child_col, d_input, stream); + num_rows = child_column->size(); // TODO add to struct column child_columns.push_back(std::move(child_column)); + column_names.back().children = names; } return {make_structs_column(num_rows, std::move(child_columns), @@ -724,7 +726,8 @@ std::pair<std::unique_ptr<column>, std::vector<std::string>> json_column_to_cudf } case json_col_t::ListColumn: { size_type num_rows = json_col.child_offsets.size(); - std::vector<std::string> cn{}; + std::vector<column_name_info> column_names{}; + column_names.emplace_back(json_col.child_columns.begin()->first); rmm::device_uvector<json_column::row_offset_t> d_offsets(num_rows, stream); cudaMemcpyAsync(d_offsets.data(), @@ -735,33 +738,63 @@ std::pair<std::unique_ptr<column>, std::vector<std::string>> json_column_to_cudf auto offsets_column = std::make_unique<column>(data_type{type_id::INT32}, num_rows, d_offsets.release()); // Create children column - auto [child_column, name] = + auto [child_column, names] = json_column_to_cudf_column(json_col.child_columns.begin()->second, d_input, stream); + column_names.back().children = names; return {make_lists_column(num_rows - 1, std::move(offsets_column), std::move(child_column), cudf::UNKNOWN_NULL_COUNT, rmm::device_buffer{}, stream), - std::move(cn)}; + std::move(column_names)}; break; } default: CUDF_FAIL("Unsupported column type, yet to be implemented"); break; } - std::vector<std::string> cn{}; - return {std::move(cudf_col), std::move(cn)}; + + return {std::move(cudf_col), std::vector<column_name_info>{}}; } -std::unique_ptr<column> parse_json_to_columns(host_span<SymbolT const> input, - rmm::cuda_stream_view stream) +table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, + rmm::cuda_stream_view stream) { // Allocate device memory for the JSON input & copy over to device rmm::device_uvector<SymbolT> d_input{input.size(), stream}; cudaMemcpyAsync( d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); + + // Get internal JSON column auto root_column = get_json_columns(input, {d_input.data(), d_input.size()}, stream); - return std::move( - json_column_to_cudf_column(root_column, {d_input.data(), d_input.size()}, stream).first); + + // Verify that we were in fact given a list of structs (or in JSON speech: an array of objects) + auto constexpr single_child_col_count = 1; + CUDF_EXPECTS(root_column.type == json_col_t::ListColumn and + root_column.child_columns.size() == single_child_col_count and + root_column.child_columns.begin()->second.type == json_col_t::StructColumn, + "Currently the nested JSON parser only supports an array of (nested) objects"); + + // Slice off the root list column, which has only a single row that contains all the structs + auto const& root_struct_col = root_column.child_columns.begin()->second; + + // Initialize meta data to be populated while recursing through the tree of columns + std::vector<std::unique_ptr<column>> out_columns; + std::vector<column_name_info> out_column_names; + + // Iterate over the struct's child columns and convert to cudf column + for (auto const& [col_name, json_col] : root_struct_col.child_columns) { + // Insert this columns name into the schema + out_column_names.emplace_back(col_name); + + // Get this JSON column's cudf column and schema info + auto [cudf_col, col_name_info] = + json_column_to_cudf_column(json_col, {d_input.data(), d_input.size()}, stream); + out_column_names.back().children = std::move(col_name_info); + out_columns.emplace_back(std::move(cudf_col)); + } + + return table_with_metadata{std::make_unique<table>(std::move(out_columns)), + {{}, out_column_names}}; } json_column get_json_columns(host_span<SymbolT const> input, diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index bf3c2703063..0776b51a437 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -348,21 +348,27 @@ TEST_F(JsonTest, ExtractColumn) std::string input = R"( [{"a":0.0, "b":1.0}, {"a":0.1, "b":1.1}, {"a":0.2, "b":1.2}] )"; // Get the JSON's tree representation - auto cudf_column = nested_json::detail::parse_json_to_columns( + auto const cudf_table = nested_json::detail::parse_json_to_columns( cudf::host_span<SymbolT const>{input.data(), input.size()}, stream_view); + auto const expected_col_count = 2; + auto const first_column_index = 0; + auto const second_column_index = 1; + EXPECT_EQ(cudf_table.tbl->num_columns(), expected_col_count); + std::cout << std::endl << "=== PARSED COLUMN ===" << std::endl; - cudf::test::print(*cudf_column); - cudf::column_view cudf_struct_view = - cudf_column->child(cudf::lists_column_view::child_column_index); + for (std::size_t col = 0; col < expected_col_count; col++) { + std::cout << std::left << std::setw(20) << cudf_table.metadata.schema_info[col].name << ": "; + cudf::test::print(cudf_table.tbl->get_column(col)); + } auto expected_col1 = cudf::test::strings_column_wrapper({"0.0", "0.1", "0.2"}); auto expected_col2 = cudf::test::strings_column_wrapper({"1.0", "1.1", "1.2"}); - cudf::column_view parsed_col1 = cudf_struct_view.child(0); + cudf::column_view parsed_col1 = cudf_table.tbl->get_column(first_column_index); CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected_col1, parsed_col1); std::cout << "*parsed_col1:\n"; cudf::test::print(parsed_col1); - cudf::column_view parsed_col2 = cudf_struct_view.child(1); + cudf::column_view parsed_col2 = cudf_table.tbl->get_column(second_column_index); CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected_col2, parsed_col2); std::cout << "*parsed_col2:\n"; cudf::test::print(parsed_col2); From a1d8901fba6a400928dc3ff0198c1cf8d26aae08 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 28 Jul 2022 05:51:48 -0700 Subject: [PATCH 68/81] removes debug print examples --- cpp/tests/io/nested_json_test.cu | 47 -------------------------------- 1 file changed, 47 deletions(-) diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index 0776b51a437..dddf97dfca2 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -289,53 +289,6 @@ TEST_F(JsonTest, TokenStream) } } -TEST_F(JsonTest, Simple) -{ - using nested_json::PdaTokenT; - using nested_json::SymbolOffsetT; - using nested_json::SymbolT; - - // Prepare cuda stream for data transfers & kernels - rmm::cuda_stream stream{}; - rmm::cuda_stream_view stream_view(stream); - - // - // std::string input = R"( ["foo", null, "bar"] )"; - // std::string input = R"( [{"a":0.0, "c":{"c0":"0.2.0"}}, {"b":1.1}] )"; - // std::string input = R"( [{"a":0.0}, {"b":1.1, "c":{"c0":"1.2.0"}}] )"; - std::string input = R"( [{"a":0.0}, {"b":1.1, "c":{"c0":[[1],null,[2]]}}] )"; - // std::string input = - // R"( [{ "col0":[{"field1": 1, "field2": 2 }, null, {"field1": 3, "field2": 4 }, {"field1": 5, - // "field2": 6 }], "col1":"foo" }] )"; - // std::string input = R"( [ {"col1": 1, "col2": 2 }, {"col1": 3, "col2": 4 }, {"col1": 5, - // "col2": 6 }] )"; std::string input = R"( [ {"col1": 1, "col2": 2 }, null, {"col1": 3, "col2": - // 4 }, - // {"col1": 5, "col2": 6 }] )"; std::string input = R"( [[1], [2], null, [3], [4]] )"; - - // String / value - // std::string input = R"( " Foobar" )"; - // std::string input = R"( 123.456 )"; - // std::string input = R"( 123.456)"; - // std::string input = R"(null)"; - // std::string input = R"( [null, [2], null, [3], [4]] )"; // <= will fail because col will be - // inferred as string/val column - - // Allocate device memory for the JSON input & copy over to device - rmm::device_uvector<SymbolT> d_input{input.size(), stream_view}; - cudaMemcpyAsync(d_input.data(), - input.data(), - input.size() * sizeof(input[0]), - cudaMemcpyHostToDevice, - stream.value()); - - // Get the JSON's tree representation - auto json_root_col = nested_json::detail::get_json_columns( - cudf::host_span<SymbolT const>{input.data(), input.size()}, d_input, stream_view); - - std::cout << input << "\n"; - print_column(input, json_root_col); -} - TEST_F(JsonTest, ExtractColumn) { using nested_json::PdaTokenT; From b9296d64b07ca69d57e048bdc8db910553e0e3b5 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 28 Jul 2022 07:57:27 -0700 Subject: [PATCH 69/81] renames lists child col name to elements --- cpp/src/io/json/nested_json_gpu.cu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index cec8c3148e7..97bb05d4955 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -802,7 +802,7 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) { // Default name for a list's child column - std::string const list_child_name = "items"; + std::string const list_child_name = "elements"; constexpr std::size_t single_item = 1; hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream}; From 5eddcc7a00e4ec73a1e99d13ed6b5cf1d4999ce4 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 28 Jul 2022 08:21:20 -0700 Subject: [PATCH 70/81] adds validity --- cpp/src/io/json/nested_json_gpu.cu | 31 +++++++++++++++++++++++++----- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 97bb05d4955..4b94d32f037 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -22,6 +22,7 @@ #include <io/utilities/hostdevice_vector.hpp> #include <cudf/column/column_factories.hpp> +#include <cudf/detail/valid_if.cuh> #include <cudf/table/table.hpp> #include <cudf/types.hpp> #include <cudf/utilities/span.hpp> @@ -673,6 +674,18 @@ void get_token_stream(device_span<SymbolT const> d_json_in, std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to_cudf_column( json_column const& json_col, device_span<SymbolT const> d_input, rmm::cuda_stream_view stream) { + auto make_validity = [](json_column const& json_col) -> std::pair<rmm::device_buffer, size_type> + { + if(json_col.current_offset == json_col.valid_count){ + return {rmm::device_buffer{}, 0}; + } + + thrust::device_vector<json_column::row_offset_t> d_validity( + json_col.validity); // TODO device_uvector. + return + cudf::detail::valid_if(d_validity.begin(), d_validity.end(), thrust::identity<bool>{}); + }; + std::unique_ptr<column> cudf_col; switch (json_col.type) { case json_col_t::StringColumn: { @@ -698,7 +711,11 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to [data = d_input.data()] __device__(auto ip) { return char_length_pair_t{data + thrust::get<0>(ip), thrust::get<1>(ip)}; }); - return {make_strings_column(d_string_data, stream), {}}; + auto str_col_ptr = make_strings_column(d_string_data, stream); + auto [result_bitmask, null_count] = + make_validity(json_col); + str_col_ptr->set_null_mask(result_bitmask, null_count); + return {std::move(str_col_ptr), {}}; break; } case json_col_t::StructColumn: { @@ -716,10 +733,12 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to child_columns.push_back(std::move(child_column)); column_names.back().children = names; } + auto [result_bitmask, null_count] = + make_validity(json_col); return {make_structs_column(num_rows, std::move(child_columns), - cudf::UNKNOWN_NULL_COUNT, - rmm::device_buffer{}, + null_count, + std::move(result_bitmask), stream), // TODO mr. column_names}; break; @@ -741,11 +760,13 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to auto [child_column, names] = json_column_to_cudf_column(json_col.child_columns.begin()->second, d_input, stream); column_names.back().children = names; + auto [result_bitmask, null_count] = + make_validity(json_col); return {make_lists_column(num_rows - 1, std::move(offsets_column), std::move(child_column), - cudf::UNKNOWN_NULL_COUNT, - rmm::device_buffer{}, + null_count, + std::move(result_bitmask), stream), std::move(column_names)}; break; From cfcd7a128dc86551c2ba8d121bcd984a354fc3a4 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Thu, 28 Jul 2022 11:13:21 -0700 Subject: [PATCH 71/81] fixes style --- cpp/src/io/json/nested_json_gpu.cu | 30 +++++++++++------------------- 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 4b94d32f037..c7378cf9f2d 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -674,16 +674,12 @@ void get_token_stream(device_span<SymbolT const> d_json_in, std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to_cudf_column( json_column const& json_col, device_span<SymbolT const> d_input, rmm::cuda_stream_view stream) { - auto make_validity = [](json_column const& json_col) -> std::pair<rmm::device_buffer, size_type> - { - if(json_col.current_offset == json_col.valid_count){ - return {rmm::device_buffer{}, 0}; - } + auto make_validity = [](json_column const& json_col) -> std::pair<rmm::device_buffer, size_type> { + if (json_col.current_offset == json_col.valid_count) { return {rmm::device_buffer{}, 0}; } thrust::device_vector<json_column::row_offset_t> d_validity( - json_col.validity); // TODO device_uvector. - return - cudf::detail::valid_if(d_validity.begin(), d_validity.end(), thrust::identity<bool>{}); + json_col.validity); // TODO device_uvector. + return cudf::detail::valid_if(d_validity.begin(), d_validity.end(), thrust::identity<bool>{}); }; std::unique_ptr<column> cudf_col; @@ -711,9 +707,8 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to [data = d_input.data()] __device__(auto ip) { return char_length_pair_t{data + thrust::get<0>(ip), thrust::get<1>(ip)}; }); - auto str_col_ptr = make_strings_column(d_string_data, stream); - auto [result_bitmask, null_count] = - make_validity(json_col); + auto str_col_ptr = make_strings_column(d_string_data, stream); + auto [result_bitmask, null_count] = make_validity(json_col); str_col_ptr->set_null_mask(result_bitmask, null_count); return {std::move(str_col_ptr), {}}; break; @@ -733,8 +728,7 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to child_columns.push_back(std::move(child_column)); column_names.back().children = names; } - auto [result_bitmask, null_count] = - make_validity(json_col); + auto [result_bitmask, null_count] = make_validity(json_col); return {make_structs_column(num_rows, std::move(child_columns), null_count, @@ -759,15 +753,13 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to // Create children column auto [child_column, names] = json_column_to_cudf_column(json_col.child_columns.begin()->second, d_input, stream); - column_names.back().children = names; - auto [result_bitmask, null_count] = - make_validity(json_col); + column_names.back().children = names; + auto [result_bitmask, null_count] = make_validity(json_col); return {make_lists_column(num_rows - 1, std::move(offsets_column), std::move(child_column), - null_count, - std::move(result_bitmask), - stream), + nul l_count, + std::move (result_bitmask), stream), std::move(column_names)}; break; } From 4c2ea7b06f8093cc36c7e8c61a39847521b3b91e Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Fri, 29 Jul 2022 01:41:22 +0530 Subject: [PATCH 72/81] minor cleanup --- cpp/src/io/json/nested_json_gpu.cu | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index c7378cf9f2d..e57a7b85c0a 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -682,12 +682,9 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to return cudf::detail::valid_if(d_validity.begin(), d_validity.end(), thrust::identity<bool>{}); }; - std::unique_ptr<column> cudf_col; switch (json_col.type) { case json_col_t::StringColumn: { - // move string_offsets to GPU. - // transform string_offsets + d_input to string_data_ptrs. - // transform string_lengths to string_data_lengths. + // move string_offsets to GPU and tranform to string column auto col_size = json_col.string_offsets.size(); using char_length_pair_t = thrust::pair<const char*, size_type>; CUDF_EXPECTS(json_col.string_offsets.size() == json_col.string_lengths.size(), @@ -714,7 +711,6 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to break; } case json_col_t::StructColumn: { - // Create empty struct column std::vector<std::unique_ptr<column>> child_columns; std::vector<column_name_info> column_names{}; size_type num_rows{-1}; @@ -724,7 +720,6 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to auto const& child_col = col.second; auto [child_column, names] = json_column_to_cudf_column(child_col, d_input, stream); num_rows = child_column->size(); - // TODO add to struct column child_columns.push_back(std::move(child_column)); column_names.back().children = names; } @@ -758,15 +753,16 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to return {make_lists_column(num_rows - 1, std::move(offsets_column), std::move(child_column), - nul l_count, - std::move (result_bitmask), stream), + null_count, + std::move(result_bitmask), + stream), std::move(column_names)}; break; } default: CUDF_FAIL("Unsupported column type, yet to be implemented"); break; } - return {std::move(cudf_col), std::vector<column_name_info>{}}; + return {}; } table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, @@ -778,7 +774,7 @@ table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); // Get internal JSON column - auto root_column = get_json_columns(input, {d_input.data(), d_input.size()}, stream); + auto root_column = get_json_columns(input, d_input, stream); // Verify that we were in fact given a list of structs (or in JSON speech: an array of objects) auto constexpr single_child_col_count = 1; @@ -800,8 +796,7 @@ table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, out_column_names.emplace_back(col_name); // Get this JSON column's cudf column and schema info - auto [cudf_col, col_name_info] = - json_column_to_cudf_column(json_col, {d_input.data(), d_input.size()}, stream); + auto [cudf_col, col_name_info] = json_column_to_cudf_column(json_col, d_input, stream); out_column_names.back().children = std::move(col_name_info); out_columns.emplace_back(std::move(cudf_col)); } From 8fc3adcf97f2d0b9931fc0c77518ea0363478bfb Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Fri, 29 Jul 2022 02:02:35 +0530 Subject: [PATCH 73/81] use device_uvector at few places --- cpp/src/io/json/nested_json_gpu.cu | 33 ++++++++++++++++-------------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index e57a7b85c0a..2a90f8404d4 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -682,19 +682,26 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to return cudf::detail::valid_if(d_validity.begin(), d_validity.end(), thrust::identity<bool>{}); }; + auto device_uvector_from_vector = [stream](auto const& vec) { + using T = typename std::decay_t<decltype(vec)>::value_type; + auto d_vec = rmm::device_uvector<T>(vec.size(), stream); + cudaMemcpyAsync( + d_vec.data(), vec.data(), vec.size() * sizeof(vec[0]), cudaMemcpyHostToDevice, stream); + return d_vec; + }; + switch (json_col.type) { case json_col_t::StringColumn: { - // move string_offsets to GPU and tranform to string column + // move string_offsets to GPU and transform to string column auto col_size = json_col.string_offsets.size(); using char_length_pair_t = thrust::pair<const char*, size_type>; CUDF_EXPECTS(json_col.string_offsets.size() == json_col.string_lengths.size(), "string offset, string length mismatch"); rmm::device_uvector<char_length_pair_t> d_string_data(col_size, stream); - thrust::device_vector<json_column::row_offset_t> d_string_offsets( - json_col.string_offsets); // TODO device_uvector. - thrust::device_vector<json_column::row_offset_t> d_string_lengths( - json_col.string_lengths); // TODO device_uvector. - // TODO: what do we do with the column names? + rmm::device_uvector<json_column::row_offset_t> d_string_offsets = + device_uvector_from_vector(json_col.string_offsets); + rmm::device_uvector<json_column::row_offset_t> d_string_lengths = + device_uvector_from_vector(json_col.string_lengths); auto offset_length_it = thrust::make_zip_iterator(d_string_offsets.begin(), d_string_lengths.begin()); thrust::transform(rmm::exec_policy(stream), @@ -737,14 +744,10 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to std::vector<column_name_info> column_names{}; column_names.emplace_back(json_col.child_columns.begin()->first); - rmm::device_uvector<json_column::row_offset_t> d_offsets(num_rows, stream); - cudaMemcpyAsync(d_offsets.data(), - json_col.child_offsets.data(), - num_rows * sizeof(json_col.child_offsets[0]), - cudaMemcpyHostToDevice, - stream); - auto offsets_column = - std::make_unique<column>(data_type{type_id::INT32}, num_rows, d_offsets.release()); + rmm::device_uvector<json_column::row_offset_t> d_offsets = + device_uvector_from_vector(json_col.child_offsets); + auto offsets_column = std::make_unique<column>( + data_type{type_id::INT32}, num_rows, d_offsets.release()); // TODO mr. // Create children column auto [child_column, names] = json_column_to_cudf_column(json_col.child_columns.begin()->second, d_input, stream); @@ -755,7 +758,7 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to std::move(child_column), null_count, std::move(result_bitmask), - stream), + stream), // TODO mr. std::move(column_names)}; break; } From c1b92136df9204e47cefe7ed31cc0854f73d3b93 Mon Sep 17 00:00:00 2001 From: Elias Stehle <3958403+elstehle@users.noreply.github.com> Date: Sat, 30 Jul 2022 04:07:03 -0700 Subject: [PATCH 74/81] fixes metadata to match parquets metadata --- cpp/src/io/json/nested_json_gpu.cu | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 2a90f8404d4..61288e96eb5 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -714,7 +714,7 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to auto str_col_ptr = make_strings_column(d_string_data, stream); auto [result_bitmask, null_count] = make_validity(json_col); str_col_ptr->set_null_mask(result_bitmask, null_count); - return {std::move(str_col_ptr), {}}; + return {std::move(str_col_ptr), {{"offsets"}, {"chars"}}}; break; } case json_col_t::StructColumn: { @@ -742,6 +742,7 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to case json_col_t::ListColumn: { size_type num_rows = json_col.child_offsets.size(); std::vector<column_name_info> column_names{}; + column_names.emplace_back("offsets"); column_names.emplace_back(json_col.child_columns.begin()->first); rmm::device_uvector<json_column::row_offset_t> d_offsets = @@ -813,7 +814,7 @@ json_column get_json_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) { // Default name for a list's child column - std::string const list_child_name = "elements"; + std::string const list_child_name = "element"; constexpr std::size_t single_item = 1; hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream}; From fd024374030411cc3c54595d02ae5436fcf2bb4d Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Mon, 1 Aug 2022 12:31:52 +0530 Subject: [PATCH 75/81] use make_device_uvector_async --- cpp/src/io/json/nested_json_gpu.cu | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 61288e96eb5..cf958644fe5 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -22,6 +22,7 @@ #include <io/utilities/hostdevice_vector.hpp> #include <cudf/column/column_factories.hpp> +#include <cudf/detail/utilities/vector_factories.hpp> #include <cudf/detail/valid_if.cuh> #include <cudf/table/table.hpp> #include <cudf/types.hpp> @@ -682,14 +683,6 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to return cudf::detail::valid_if(d_validity.begin(), d_validity.end(), thrust::identity<bool>{}); }; - auto device_uvector_from_vector = [stream](auto const& vec) { - using T = typename std::decay_t<decltype(vec)>::value_type; - auto d_vec = rmm::device_uvector<T>(vec.size(), stream); - cudaMemcpyAsync( - d_vec.data(), vec.data(), vec.size() * sizeof(vec[0]), cudaMemcpyHostToDevice, stream); - return d_vec; - }; - switch (json_col.type) { case json_col_t::StringColumn: { // move string_offsets to GPU and transform to string column @@ -699,9 +692,9 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to "string offset, string length mismatch"); rmm::device_uvector<char_length_pair_t> d_string_data(col_size, stream); rmm::device_uvector<json_column::row_offset_t> d_string_offsets = - device_uvector_from_vector(json_col.string_offsets); + cudf::detail::make_device_uvector_async(json_col.string_offsets, stream); rmm::device_uvector<json_column::row_offset_t> d_string_lengths = - device_uvector_from_vector(json_col.string_lengths); + cudf::detail::make_device_uvector_async(json_col.string_lengths, stream); auto offset_length_it = thrust::make_zip_iterator(d_string_offsets.begin(), d_string_lengths.begin()); thrust::transform(rmm::exec_policy(stream), @@ -746,7 +739,7 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to column_names.emplace_back(json_col.child_columns.begin()->first); rmm::device_uvector<json_column::row_offset_t> d_offsets = - device_uvector_from_vector(json_col.child_offsets); + cudf::detail::make_device_uvector_async(json_col.child_offsets, stream); auto offsets_column = std::make_unique<column>( data_type{type_id::INT32}, num_rows, d_offsets.release()); // TODO mr. // Create children column @@ -773,9 +766,7 @@ table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, rmm::cuda_stream_view stream) { // Allocate device memory for the JSON input & copy over to device - rmm::device_uvector<SymbolT> d_input{input.size(), stream}; - cudaMemcpyAsync( - d_input.data(), input.data(), input.size() * sizeof(input[0]), cudaMemcpyHostToDevice, stream); + rmm::device_uvector<SymbolT> d_input = cudf::detail::make_device_uvector_async(input, stream); // Get internal JSON column auto root_column = get_json_columns(input, d_input, stream); From f5810f80ad88a013a6ebc43df24b00e1935074cc Mon Sep 17 00:00:00 2001 From: Karthikeyan <6488848+karthikeyann@users.noreply.github.com> Date: Mon, 1 Aug 2022 21:51:10 +0530 Subject: [PATCH 76/81] Apply suggestions from code review (nvdbaranec) Co-authored-by: nvdbaranec <56695930+nvdbaranec@users.noreply.github.com> --- cpp/src/io/json/nested_json_gpu.cu | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index cf958644fe5..417614c8ae9 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -686,7 +686,7 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to switch (json_col.type) { case json_col_t::StringColumn: { // move string_offsets to GPU and transform to string column - auto col_size = json_col.string_offsets.size(); + auto const col_size = json_col.string_offsets.size(); using char_length_pair_t = thrust::pair<const char*, size_type>; CUDF_EXPECTS(json_col.string_offsets.size() == json_col.string_lengths.size(), "string offset, string length mismatch"); @@ -946,7 +946,7 @@ json_column get_json_columns(host_span<SymbolT const> input, // String | List => null // String | Struct => null // String | String => valid - bool is_valid = (column->type == row_type); + bool const is_valid = (column->type == row_type); column->validity.push_back(is_valid); column->valid_count += (is_valid) ? 1U : 0U; column->string_offsets.push_back(string_offset); @@ -1150,7 +1150,7 @@ json_column get_json_columns(host_span<SymbolT const> input, json_column* selected_col = get_selected_column(current_data_path); // Get the row offset at which to insert - auto target_row_index = get_target_row_index(current_data_path, selected_col); + auto const target_row_index = get_target_row_index(current_data_path, selected_col); // Increment parent's child count and insert this struct node into the data path current_data_path.top().num_children++; @@ -1189,7 +1189,7 @@ json_column get_json_columns(host_span<SymbolT const> input, json_column* selected_col = get_selected_column(current_data_path); // Get the row offset at which to insert - auto target_row_index = get_target_row_index(current_data_path, selected_col); + auto const target_row_index = get_target_row_index(current_data_path, selected_col); // Increment parent's child count and insert this struct node into the data path current_data_path.top().num_children++; @@ -1259,7 +1259,7 @@ json_column get_json_columns(host_span<SymbolT const> input, json_column* selected_col = get_selected_column(current_data_path); // Get the row offset at which to insert - auto target_row_index = get_target_row_index(current_data_path, selected_col); + auto const target_row_index = get_target_row_index(current_data_path, selected_col); current_data_path.top().num_children++; From 342d3c3d7065f5c8b2301e546827be52eb66a27b Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Tue, 2 Aug 2022 00:03:42 +0530 Subject: [PATCH 77/81] added mr --- cpp/src/io/json/nested_json.hpp | 7 +++-- cpp/src/io/json/nested_json_gpu.cu | 47 +++++++++++++++++------------- 2 files changed, 32 insertions(+), 22 deletions(-) diff --git a/cpp/src/io/json/nested_json.hpp b/cpp/src/io/json/nested_json.hpp index 75987215e7e..a3f5c6d0cb2 100644 --- a/cpp/src/io/json/nested_json.hpp +++ b/cpp/src/io/json/nested_json.hpp @@ -17,6 +17,7 @@ #pragma once #include <cudf/io/types.hpp> +#include <cudf/utilities/default_stream.hpp> #include <cudf/types.hpp> #include <cudf/utilities/span.hpp> @@ -260,8 +261,10 @@ json_column get_json_columns(host_span<SymbolT const> input, * @param stream The CUDA stream to which kernels are dispatched * @return cudf::column of the given input */ -table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, - rmm::cuda_stream_view stream); +table_with_metadata parse_json_to_columns( + host_span<SymbolT const> input, + rmm::cuda_stream_view stream = cudf::default_stream_value, + rmm::mr::device_memory_resource* mr = rmm::mr::get_current_device_resource()); } // namespace detail diff --git a/cpp/src/io/json/nested_json_gpu.cu b/cpp/src/io/json/nested_json_gpu.cu index 417614c8ae9..9f89879e101 100644 --- a/cpp/src/io/json/nested_json_gpu.cu +++ b/cpp/src/io/json/nested_json_gpu.cu @@ -673,20 +673,25 @@ void get_token_stream(device_span<SymbolT const> d_json_in, } std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to_cudf_column( - json_column const& json_col, device_span<SymbolT const> d_input, rmm::cuda_stream_view stream) + json_column const& json_col, + device_span<SymbolT const> d_input, + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) { - auto make_validity = [](json_column const& json_col) -> std::pair<rmm::device_buffer, size_type> { + auto make_validity = + [stream, mr](json_column const& json_col) -> std::pair<rmm::device_buffer, size_type> { if (json_col.current_offset == json_col.valid_count) { return {rmm::device_buffer{}, 0}; } thrust::device_vector<json_column::row_offset_t> d_validity( json_col.validity); // TODO device_uvector. - return cudf::detail::valid_if(d_validity.begin(), d_validity.end(), thrust::identity<bool>{}); + return cudf::detail::valid_if( + d_validity.begin(), d_validity.end(), thrust::identity<bool>{}, stream, mr); }; switch (json_col.type) { case json_col_t::StringColumn: { // move string_offsets to GPU and transform to string column - auto const col_size = json_col.string_offsets.size(); + auto const col_size = json_col.string_offsets.size(); using char_length_pair_t = thrust::pair<const char*, size_type>; CUDF_EXPECTS(json_col.string_offsets.size() == json_col.string_lengths.size(), "string offset, string length mismatch"); @@ -704,7 +709,7 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to [data = d_input.data()] __device__(auto ip) { return char_length_pair_t{data + thrust::get<0>(ip), thrust::get<1>(ip)}; }); - auto str_col_ptr = make_strings_column(d_string_data, stream); + auto str_col_ptr = make_strings_column(d_string_data, stream, mr); auto [result_bitmask, null_count] = make_validity(json_col); str_col_ptr->set_null_mask(result_bitmask, null_count); return {std::move(str_col_ptr), {{"offsets"}, {"chars"}}}; @@ -718,18 +723,18 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to for (auto const& col : json_col.child_columns) { column_names.emplace_back(col.first); auto const& child_col = col.second; - auto [child_column, names] = json_column_to_cudf_column(child_col, d_input, stream); - num_rows = child_column->size(); + auto [child_column, names] = json_column_to_cudf_column(child_col, d_input, stream, mr); + if (num_rows < 0) num_rows = child_column->size(); + CUDF_EXPECTS(num_rows == child_column->size(), + "All children columns must have the same size"); child_columns.push_back(std::move(child_column)); column_names.back().children = names; } auto [result_bitmask, null_count] = make_validity(json_col); - return {make_structs_column(num_rows, - std::move(child_columns), - null_count, - std::move(result_bitmask), - stream), // TODO mr. - column_names}; + return { + make_structs_column( + num_rows, std::move(child_columns), null_count, std::move(result_bitmask), stream, mr), + column_names}; break; } case json_col_t::ListColumn: { @@ -739,12 +744,12 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to column_names.emplace_back(json_col.child_columns.begin()->first); rmm::device_uvector<json_column::row_offset_t> d_offsets = - cudf::detail::make_device_uvector_async(json_col.child_offsets, stream); - auto offsets_column = std::make_unique<column>( - data_type{type_id::INT32}, num_rows, d_offsets.release()); // TODO mr. + cudf::detail::make_device_uvector_async(json_col.child_offsets, stream, mr); + auto offsets_column = + std::make_unique<column>(data_type{type_id::INT32}, num_rows, d_offsets.release()); // Create children column auto [child_column, names] = - json_column_to_cudf_column(json_col.child_columns.begin()->second, d_input, stream); + json_column_to_cudf_column(json_col.child_columns.begin()->second, d_input, stream, mr); column_names.back().children = names; auto [result_bitmask, null_count] = make_validity(json_col); return {make_lists_column(num_rows - 1, @@ -752,7 +757,8 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to std::move(child_column), null_count, std::move(result_bitmask), - stream), // TODO mr. + stream, + mr), std::move(column_names)}; break; } @@ -763,7 +769,8 @@ std::pair<std::unique_ptr<column>, std::vector<column_name_info>> json_column_to } table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, - rmm::cuda_stream_view stream) + rmm::cuda_stream_view stream, + rmm::mr::device_memory_resource* mr) { // Allocate device memory for the JSON input & copy over to device rmm::device_uvector<SymbolT> d_input = cudf::detail::make_device_uvector_async(input, stream); @@ -791,7 +798,7 @@ table_with_metadata parse_json_to_columns(host_span<SymbolT const> input, out_column_names.emplace_back(col_name); // Get this JSON column's cudf column and schema info - auto [cudf_col, col_name_info] = json_column_to_cudf_column(json_col, d_input, stream); + auto [cudf_col, col_name_info] = json_column_to_cudf_column(json_col, d_input, stream, mr); out_column_names.back().children = std::move(col_name_info); out_columns.emplace_back(std::move(cudf_col)); } From f6a531ae139cdd4c0a7f173440775db1b50ddf89 Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Wed, 3 Aug 2022 02:18:38 +0530 Subject: [PATCH 78/81] add utf fail, pass test cases --- cpp/tests/io/nested_json_test.cu | 42 ++++++++++++++++++++++++++++++-- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu index dddf97dfca2..f335ad360c1 100644 --- a/cpp/tests/io/nested_json_test.cu +++ b/cpp/tests/io/nested_json_test.cu @@ -291,8 +291,6 @@ TEST_F(JsonTest, TokenStream) TEST_F(JsonTest, ExtractColumn) { - using nested_json::PdaTokenT; - using nested_json::SymbolOffsetT; using nested_json::SymbolT; // Prepare cuda stream for data transfers & kernels @@ -326,3 +324,43 @@ TEST_F(JsonTest, ExtractColumn) std::cout << "*parsed_col2:\n"; cudf::test::print(parsed_col2); } + +TEST_F(JsonTest, UTF_JSON) +{ + // Prepare cuda stream for data transfers & kernels + rmm::cuda_stream stream{}; + rmm::cuda_stream_view stream_view(stream); + + // Only ASCII string + std::string ascii_pass = R"([ + {"a":1,"b":2,"c":[3], "d": {}}, + {"a":1,"b":4.0,"c":[], "d": {"year":1882,"author": "Bharathi"}}, + {"a":1,"b":6.0,"c":[5, 7], "d": null}, + {"a":1,"b":8.0,"c":null, "d": {}}, + {"a":1,"b":null,"c":null}, + {"a":1,"b":Infinity,"c":[null], "d": {"year":-600,"author": "Kaniyan"}}])"; + + CUDF_EXPECT_NO_THROW(nested_json::detail::parse_json_to_columns(ascii_pass, stream_view)); + + // utf-8 string that fails parsing. + std::string utf_failed = R"([ + {"a":1,"b":2,"c":[3], "d": {}}, + {"a":1,"b":4.0,"c":[], "d": {"year":1882,"author": "Bharathi"}}, + {"a":1,"b":6.0,"c":[5, 7], "d": null}, + {"a":1,"b":8.0,"c":null, "d": {}}, + {"a":1,"b":null,"c":null}, + {"a":1,"b":Infinity,"c":[null], "d": {"year":-600,"author": "filip ʒakotɛ"}}])"; + CUDF_EXPECT_THROW_MESSAGE(nested_json::detail::parse_json_to_columns(utf_failed, stream_view), + "Parser encountered an invalid format."); + + // utf-8 string that passes parsing. + std::string utf_pass = R"([ + {"a":1,"b":2,"c":[3], "d": {}}, + {"a":1,"b":4.0,"c":[], "d": {"year":1882,"author": "Bharathi"}}, + {"a":1,"b":6.0,"c":[5, 7], "d": null}, + {"a":1,"b":8.0,"c":null, "d": {}}, + {"a":1,"b":null,"c":null}, + {"a":1,"b":Infinity,"c":[null], "d": {"year":-600,"author": "Kaniyan"}}, + {"a":1,"b":NaN,"c":[null, null], "d": {"year": 2, "author": "filip ʒakotɛ"}}])"; + CUDF_EXPECT_NO_THROW(nested_json::detail::parse_json_to_columns(utf_pass, stream_view)); +} From a141110e4b979a3184ee87853cfd9324e7f6d278 Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Thu, 4 Aug 2022 21:57:34 +0530 Subject: [PATCH 79/81] add NESTED_JSON_NVBENCH --- cpp/benchmarks/CMakeLists.txt | 1 + cpp/benchmarks/io/json/nested_json.cpp | 89 ++++++++++++++++++++++++++ 2 files changed, 90 insertions(+) create mode 100644 cpp/benchmarks/io/json/nested_json.cpp diff --git a/cpp/benchmarks/CMakeLists.txt b/cpp/benchmarks/CMakeLists.txt index bf1062e0449..e6b3f1a7acf 100644 --- a/cpp/benchmarks/CMakeLists.txt +++ b/cpp/benchmarks/CMakeLists.txt @@ -289,6 +289,7 @@ ConfigureBench( # * json benchmark ------------------------------------------------------------------- ConfigureBench(JSON_BENCH string/json.cu) ConfigureNVBench(FST_NVBENCH io/fst.cu) +ConfigureNVBench(NESTED_JSON_NVBENCH io/json/nested_json.cpp) # ################################################################################################## # * io benchmark --------------------------------------------------------------------- diff --git a/cpp/benchmarks/io/json/nested_json.cpp b/cpp/benchmarks/io/json/nested_json.cpp new file mode 100644 index 00000000000..cf3ac8a8b3e --- /dev/null +++ b/cpp/benchmarks/io/json/nested_json.cpp @@ -0,0 +1,89 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed 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 <benchmarks/common/generate_input.hpp> +#include <benchmarks/fixture/rmm_pool_raii.hpp> +#include <nvbench/nvbench.cuh> + +#include <io/json/nested_json.hpp> +#include <io/utilities/hostdevice_vector.hpp> //TODO find better replacement + +#include <tests/io/fst/common.hpp> + +#include <cudf/scalar/scalar_factories.hpp> +#include <cudf/strings/repeat_strings.hpp> +#include <cudf/types.hpp> + +#include <rmm/cuda_stream.hpp> +#include <rmm/cuda_stream_view.hpp> + +#include <cstdlib> + +namespace cudf { +namespace { +auto make_test_json_data(size_type string_size, rmm::cuda_stream_view stream) +{ + // Test input + std::string input = R"( + {"a":1,"b":2,"c":[3], "d": {}}, + {"a":1,"b":4.0,"c":[], "d": {"year":1882,"author": "Bharathi"}}, + {"a":1,"b":6.0,"c":[5, 7], "d": null}, + {"a":1,"b":null,"c":null}, + { + "a" : 1 + }, + {"a":1,"b":Infinity,"c":[null], "d": {"year":-600,"author": "Kaniyan"}}, + {"a": 1, "b": 8.0, "d": { "author": "Jean-Jacques Rousseau"}},)"; + + const size_type repeat_times = string_size / input.size(); + + auto d_input_scalar = cudf::make_string_scalar(input, stream); + auto& d_string_scalar = static_cast<cudf::string_scalar&>(*d_input_scalar); + auto d_scalar = cudf::strings::repeat_string(d_string_scalar, repeat_times); + auto& d_input = static_cast<cudf::scalar_type_t<std::string>&>(*d_scalar); + auto generated_json = std::string(d_input); + generated_json[0] = '['; + generated_json[generated_json.size() - 1] = ']'; + return generated_json; +} +} // namespace + +void BM_NESTED_JSON(nvbench::state& state) +{ + // TODO: to be replaced by nvbench fixture once it's ready + cudf::rmm_pool_raii rmm_pool; + + auto const string_size{size_type(state.get_int64("string_size"))}; + // Prepare cuda stream for data transfers & kernels + rmm::cuda_stream stream{}; + rmm::cuda_stream_view stream_view(stream); + + auto input = make_test_json_data(string_size, stream_view); + state.add_element_count(input.size()); + + // Run algorithm + state.set_cuda_stream(nvbench::make_cuda_stream_view(stream.value())); + state.exec(nvbench::exec_tag::sync, [&](nvbench::launch& launch) { + // Allocate device-side temporary storage & run algorithm + cudf::io::json::detail::parse_json_to_columns(input, stream_view); + }); +} + +NVBENCH_BENCH(BM_NESTED_JSON) + .set_name("nested_json_gpu_parser") + .add_int64_power_of_two_axis("string_size", nvbench::range(20, 31, 1)); + +} // namespace cudf From ba3483f07f9ece7acbe87bc8e8c442c9dbb7bddd Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Wed, 24 Aug 2022 00:52:54 +0530 Subject: [PATCH 80/81] remove merge missed files --- cpp/tests/io/nested_json_test.cu | 366 ------------------------------- 1 file changed, 366 deletions(-) delete mode 100644 cpp/tests/io/nested_json_test.cu diff --git a/cpp/tests/io/nested_json_test.cu b/cpp/tests/io/nested_json_test.cu deleted file mode 100644 index f335ad360c1..00000000000 --- a/cpp/tests/io/nested_json_test.cu +++ /dev/null @@ -1,366 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed 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 "cudf_test/column_wrapper.hpp" - -#include <io/json/nested_json.hpp> -#include <io/utilities/hostdevice_vector.hpp> - -#include <cudf_test/base_fixture.hpp> -#include <cudf_test/column_utilities.hpp> -#include <cudf_test/cudf_gtest.hpp> - -#include <cudf/lists/lists_column_view.hpp> - -#include <rmm/cuda_stream.hpp> -#include <rmm/cuda_stream_view.hpp> - -#include <string> - -namespace nested_json = cudf::io::json; - -namespace { -// Forward declaration -void print_column(std::string const& input, - nested_json::json_column const& column, - uint32_t indent = 0); - -/** - * @brief Helper to generate indentation - */ -std::string pad(uint32_t indent = 0) -{ - std::string pad{}; - if (indent > 0) pad.insert(pad.begin(), indent, ' '); - return pad; -} - -/** - * @brief Prints a string column. - */ -void print_json_string_col(std::string const& input, - nested_json::json_column const& column, - uint32_t indent = 0) -{ - for (std::size_t i = 0; i < column.string_offsets.size(); i++) { - std::cout << pad(indent) << i << ": [" << (column.validity[i] ? "1" : "0") << "] '" - << input.substr(column.string_offsets[i], column.string_lengths[i]) << "'\n"; - } -} - -/** - * @brief Prints a list column. - */ -void print_json_list_col(std::string const& input, - nested_json::json_column const& column, - uint32_t indent = 0) -{ - std::cout << pad(indent) << " [LIST]\n"; - std::cout << pad(indent) << " -> num. child-columns: " << column.child_columns.size() << "\n"; - std::cout << pad(indent) << " -> num. rows: " << column.current_offset << "\n"; - std::cout << pad(indent) << " -> num. valid: " << column.valid_count << "\n"; - std::cout << pad(indent) << " offsets[]: " - << "\n"; - for (std::size_t i = 0; i < column.child_offsets.size() - 1; i++) { - std::cout << pad(indent + 2) << i << ": [" << (column.validity[i] ? "1" : "0") << "] [" - << column.child_offsets[i] << ", " << column.child_offsets[i + 1] << ")\n"; - } - if (column.child_columns.size() > 0) { - std::cout << pad(indent) << column.child_columns.begin()->first << "[]: " - << "\n"; - print_column(input, column.child_columns.begin()->second, indent + 2); - } -} - -/** - * @brief Prints a struct column. - */ -void print_json_struct_col(std::string const& input, - nested_json::json_column const& column, - uint32_t indent = 0) -{ - std::cout << pad(indent) << " [STRUCT]\n"; - std::cout << pad(indent) << " -> num. child-columns: " << column.child_columns.size() << "\n"; - std::cout << pad(indent) << " -> num. rows: " << column.current_offset << "\n"; - std::cout << pad(indent) << " -> num. valid: " << column.valid_count << "\n"; - std::cout << pad(indent) << " -> validity[]: " - << "\n"; - for (std::size_t i = 0; i < column.current_offset; i++) { - std::cout << pad(indent + 2) << i << ": [" << (column.validity[i] ? "1" : "0") << "]\n"; - } - auto it = std::begin(column.child_columns); - for (std::size_t i = 0; i < column.child_columns.size(); i++) { - std::cout << pad(indent + 2) << "child #" << i << " '" << it->first << "'[] \n"; - print_column(input, it->second, indent + 2); - it++; - } -} - -/** - * @brief Prints the column's data and recurses through and prints all the child columns. - */ -void print_column(std::string const& input, nested_json::json_column const& column, uint32_t indent) -{ - switch (column.type) { - case nested_json::json_col_t::StringColumn: print_json_string_col(input, column, indent); break; - case nested_json::json_col_t::ListColumn: print_json_list_col(input, column, indent); break; - case nested_json::json_col_t::StructColumn: print_json_struct_col(input, column, indent); break; - case nested_json::json_col_t::Unknown: std::cout << pad(indent) << "[UNKNOWN]\n"; break; - default: break; - } -} -} // namespace - -// Base test fixture for tests -struct JsonTest : public cudf::test::BaseFixture { -}; - -TEST_F(JsonTest, StackContext) -{ - // Type used to represent the atomic symbol type used within the finite-state machine - using SymbolT = char; - using StackSymbolT = char; - - // Prepare cuda stream for data transfers & kernels - rmm::cuda_stream stream{}; - rmm::cuda_stream_view stream_view(stream); - - // Test input - std::string input = R"( [{)" - R"("category": "reference",)" - R"("index:": [4,12,42],)" - R"("author": "Nigel Rees",)" - R"("title": "[Sayings of the Century]",)" - R"("price": 8.95)" - R"(}, )" - R"({)" - R"("category": "reference",)" - R"("index": [4,{},null,{"a":[{ }, {}] } ],)" - R"("author": "Nigel Rees",)" - R"("title": "{}\\\"[], <=semantic-symbols-string\\\\",)" - R"("price": 8.95)" - R"(}] )"; - - // Prepare input & output buffers - rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); - hostdevice_vector<StackSymbolT> stack_context(input.size(), stream_view); - - ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync(d_input.data(), - input.data(), - input.size() * sizeof(SymbolT), - cudaMemcpyHostToDevice, - stream.value())); - - // Run algorithm - nested_json::detail::get_stack_context(d_input, stack_context.device_ptr(), stream_view); - - // Copy back the results - stack_context.device_to_host(stream_view); - - // Make sure we copied back the stack context - stream_view.synchronize(); - - std::vector<char> golden_stack_context{ - '_', '_', '_', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '[', '[', '[', '[', '[', '[', '[', '[', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', '[', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '[', '[', '[', '{', '[', '[', '[', '[', '[', '[', '[', '{', - '{', '{', '{', '{', '[', '{', '{', '[', '[', '[', '{', '[', '{', '{', '[', '[', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', - '{', '{', '{', '{', '{', '{', '{', '{', '{', '{', '[', '_'}; - - ASSERT_EQ(golden_stack_context.size(), stack_context.size()); - CUDF_TEST_EXPECT_VECTOR_EQUAL(golden_stack_context, stack_context, stack_context.size()); -} - -TEST_F(JsonTest, TokenStream) -{ - using nested_json::PdaTokenT; - using nested_json::SymbolOffsetT; - using nested_json::SymbolT; - - constexpr std::size_t single_item = 1; - - // Prepare cuda stream for data transfers & kernels - rmm::cuda_stream stream{}; - rmm::cuda_stream_view stream_view(stream); - - // Test input - std::string input = R"( [{)" - R"("category": "reference",)" - R"("index:": [4,12,42],)" - R"("author": "Nigel Rees",)" - R"("title": "[Sayings of the Century]",)" - R"("price": 8.95)" - R"(}, )" - R"({)" - R"("category": "reference",)" - R"("index": [4,{},null,{"a":[{ }, {}] } ],)" - R"("author": "Nigel Rees",)" - R"("title": "{}[], <=semantic-symbols-string",)" - R"("price": 8.95)" - R"(}] )"; - - // Prepare input & output buffers - rmm::device_uvector<SymbolT> d_input(input.size(), stream_view); - - ASSERT_CUDA_SUCCEEDED(cudaMemcpyAsync(d_input.data(), - input.data(), - input.size() * sizeof(SymbolT), - cudaMemcpyHostToDevice, - stream.value())); - - hostdevice_vector<PdaTokenT> tokens_gpu{input.size(), stream_view}; - hostdevice_vector<SymbolOffsetT> token_indices_gpu{input.size(), stream_view}; - hostdevice_vector<SymbolOffsetT> num_tokens_out{single_item, stream_view}; - - // Parse the JSON and get the token stream - nested_json::detail::get_token_stream(d_input, - tokens_gpu.device_ptr(), - token_indices_gpu.device_ptr(), - num_tokens_out.device_ptr(), - stream_view); - - // Copy back the number of tokens that were written - num_tokens_out.device_to_host(stream_view); - tokens_gpu.device_to_host(stream_view); - token_indices_gpu.device_to_host(stream_view); - - // Make sure we copied back all relevant data - stream_view.synchronize(); - - // Golden token stream sample - using token_t = nested_json::token_t; - std::vector<std::pair<std::size_t, nested_json::PdaTokenT>> golden_token_stream = { - {2, token_t::ListBegin}, {3, token_t::StructBegin}, {4, token_t::FieldNameBegin}, - {13, token_t::FieldNameEnd}, {16, token_t::StringBegin}, {26, token_t::StringEnd}, - {28, token_t::FieldNameBegin}, {35, token_t::FieldNameEnd}, {38, token_t::ListBegin}, - {39, token_t::ValueBegin}, {40, token_t::ValueEnd}, {41, token_t::ValueBegin}, - {43, token_t::ValueEnd}, {44, token_t::ValueBegin}, {46, token_t::ValueEnd}, - {46, token_t::ListEnd}, {48, token_t::FieldNameBegin}, {55, token_t::FieldNameEnd}, - {58, token_t::StringBegin}, {69, token_t::StringEnd}, {71, token_t::FieldNameBegin}, - {77, token_t::FieldNameEnd}, {80, token_t::StringBegin}, {105, token_t::StringEnd}, - {107, token_t::FieldNameBegin}, {113, token_t::FieldNameEnd}, {116, token_t::ValueBegin}, - {120, token_t::ValueEnd}, {120, token_t::StructEnd}, {124, token_t::StructBegin}, - {125, token_t::FieldNameBegin}, {134, token_t::FieldNameEnd}, {137, token_t::StringBegin}, - {147, token_t::StringEnd}, {149, token_t::FieldNameBegin}, {155, token_t::FieldNameEnd}, - {158, token_t::ListBegin}, {159, token_t::ValueBegin}, {160, token_t::ValueEnd}, - {161, token_t::StructBegin}, {162, token_t::StructEnd}, {164, token_t::ValueBegin}, - {168, token_t::ValueEnd}, {169, token_t::StructBegin}, {170, token_t::FieldNameBegin}, - {172, token_t::FieldNameEnd}, {174, token_t::ListBegin}, {175, token_t::StructBegin}, - {177, token_t::StructEnd}, {180, token_t::StructBegin}, {181, token_t::StructEnd}, - {182, token_t::ListEnd}, {184, token_t::StructEnd}, {186, token_t::ListEnd}, - {188, token_t::FieldNameBegin}, {195, token_t::FieldNameEnd}, {198, token_t::StringBegin}, - {209, token_t::StringEnd}, {211, token_t::FieldNameBegin}, {217, token_t::FieldNameEnd}, - {220, token_t::StringBegin}, {252, token_t::StringEnd}, {254, token_t::FieldNameBegin}, - {260, token_t::FieldNameEnd}, {263, token_t::ValueBegin}, {267, token_t::ValueEnd}, - {267, token_t::StructEnd}, {268, token_t::ListEnd}}; - - // Verify the number of tokens matches - ASSERT_EQ(golden_token_stream.size(), num_tokens_out[0]); - - for (std::size_t i = 0; i < num_tokens_out[0]; i++) { - // Ensure the index the tokens are pointing to do match - EXPECT_EQ(golden_token_stream[i].first, token_indices_gpu[i]) << "Mismatch at #" << i; - - // Ensure the token category is correct - EXPECT_EQ(golden_token_stream[i].second, tokens_gpu[i]) << "Mismatch at #" << i; - } -} - -TEST_F(JsonTest, ExtractColumn) -{ - using nested_json::SymbolT; - - // Prepare cuda stream for data transfers & kernels - rmm::cuda_stream stream{}; - rmm::cuda_stream_view stream_view(stream); - - std::string input = R"( [{"a":0.0, "b":1.0}, {"a":0.1, "b":1.1}, {"a":0.2, "b":1.2}] )"; - // Get the JSON's tree representation - auto const cudf_table = nested_json::detail::parse_json_to_columns( - cudf::host_span<SymbolT const>{input.data(), input.size()}, stream_view); - - auto const expected_col_count = 2; - auto const first_column_index = 0; - auto const second_column_index = 1; - EXPECT_EQ(cudf_table.tbl->num_columns(), expected_col_count); - - std::cout << std::endl << "=== PARSED COLUMN ===" << std::endl; - for (std::size_t col = 0; col < expected_col_count; col++) { - std::cout << std::left << std::setw(20) << cudf_table.metadata.schema_info[col].name << ": "; - cudf::test::print(cudf_table.tbl->get_column(col)); - } - - auto expected_col1 = cudf::test::strings_column_wrapper({"0.0", "0.1", "0.2"}); - auto expected_col2 = cudf::test::strings_column_wrapper({"1.0", "1.1", "1.2"}); - cudf::column_view parsed_col1 = cudf_table.tbl->get_column(first_column_index); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected_col1, parsed_col1); - std::cout << "*parsed_col1:\n"; - cudf::test::print(parsed_col1); - cudf::column_view parsed_col2 = cudf_table.tbl->get_column(second_column_index); - CUDF_TEST_EXPECT_COLUMNS_EQUAL(expected_col2, parsed_col2); - std::cout << "*parsed_col2:\n"; - cudf::test::print(parsed_col2); -} - -TEST_F(JsonTest, UTF_JSON) -{ - // Prepare cuda stream for data transfers & kernels - rmm::cuda_stream stream{}; - rmm::cuda_stream_view stream_view(stream); - - // Only ASCII string - std::string ascii_pass = R"([ - {"a":1,"b":2,"c":[3], "d": {}}, - {"a":1,"b":4.0,"c":[], "d": {"year":1882,"author": "Bharathi"}}, - {"a":1,"b":6.0,"c":[5, 7], "d": null}, - {"a":1,"b":8.0,"c":null, "d": {}}, - {"a":1,"b":null,"c":null}, - {"a":1,"b":Infinity,"c":[null], "d": {"year":-600,"author": "Kaniyan"}}])"; - - CUDF_EXPECT_NO_THROW(nested_json::detail::parse_json_to_columns(ascii_pass, stream_view)); - - // utf-8 string that fails parsing. - std::string utf_failed = R"([ - {"a":1,"b":2,"c":[3], "d": {}}, - {"a":1,"b":4.0,"c":[], "d": {"year":1882,"author": "Bharathi"}}, - {"a":1,"b":6.0,"c":[5, 7], "d": null}, - {"a":1,"b":8.0,"c":null, "d": {}}, - {"a":1,"b":null,"c":null}, - {"a":1,"b":Infinity,"c":[null], "d": {"year":-600,"author": "filip ʒakotɛ"}}])"; - CUDF_EXPECT_THROW_MESSAGE(nested_json::detail::parse_json_to_columns(utf_failed, stream_view), - "Parser encountered an invalid format."); - - // utf-8 string that passes parsing. - std::string utf_pass = R"([ - {"a":1,"b":2,"c":[3], "d": {}}, - {"a":1,"b":4.0,"c":[], "d": {"year":1882,"author": "Bharathi"}}, - {"a":1,"b":6.0,"c":[5, 7], "d": null}, - {"a":1,"b":8.0,"c":null, "d": {}}, - {"a":1,"b":null,"c":null}, - {"a":1,"b":Infinity,"c":[null], "d": {"year":-600,"author": "Kaniyan"}}, - {"a":1,"b":NaN,"c":[null, null], "d": {"year": 2, "author": "filip ʒakotɛ"}}])"; - CUDF_EXPECT_NO_THROW(nested_json::detail::parse_json_to_columns(utf_pass, stream_view)); -} From cad106074ecd2d7e2a96495a98ea7226361dc3db Mon Sep 17 00:00:00 2001 From: Karthikeyan Natarajan <karthikeyann@users.noreply.github.com> Date: Wed, 24 Aug 2022 00:59:20 +0530 Subject: [PATCH 81/81] address review comments --- cpp/benchmarks/io/json/nested_json.cpp | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/cpp/benchmarks/io/json/nested_json.cpp b/cpp/benchmarks/io/json/nested_json.cpp index c8c295ee382..34b512b5f43 100644 --- a/cpp/benchmarks/io/json/nested_json.cpp +++ b/cpp/benchmarks/io/json/nested_json.cpp @@ -16,10 +16,10 @@ #include <benchmarks/common/generate_input.hpp> #include <benchmarks/fixture/rmm_pool_raii.hpp> + #include <nvbench/nvbench.cuh> #include <io/json/nested_json.hpp> -#include <io/utilities/hostdevice_vector.hpp> //TODO find better replacement #include <tests/io/fst/common.hpp> @@ -27,9 +27,6 @@ #include <cudf/strings/repeat_strings.hpp> #include <cudf/types.hpp> -#include <rmm/cuda_stream.hpp> -#include <rmm/cuda_stream_view.hpp> - #include <cstdlib> namespace cudf { @@ -54,9 +51,10 @@ auto make_test_json_data(size_type string_size, rmm::cuda_stream_view stream) auto& d_string_scalar = static_cast<cudf::string_scalar&>(*d_input_scalar); auto d_scalar = cudf::strings::repeat_string(d_string_scalar, repeat_times); auto& d_input = static_cast<cudf::scalar_type_t<std::string>&>(*d_scalar); - auto generated_json = std::string(d_input); - generated_json[0] = '['; - generated_json[generated_json.size() - 1] = ']'; + + auto generated_json = std::string(d_input); + generated_json.front() = '['; + generated_json.back() = ']'; return generated_json; } } // namespace @@ -67,18 +65,15 @@ void BM_NESTED_JSON(nvbench::state& state) cudf::rmm_pool_raii rmm_pool; auto const string_size{size_type(state.get_int64("string_size"))}; - // Prepare cuda stream for data transfers & kernels - rmm::cuda_stream stream{}; - rmm::cuda_stream_view stream_view(stream); - auto input = make_test_json_data(string_size, stream_view); + auto input = make_test_json_data(string_size, cudf::default_stream_value); state.add_element_count(input.size()); // Run algorithm - state.set_cuda_stream(nvbench::make_cuda_stream_view(stream.value())); + state.set_cuda_stream(nvbench::make_cuda_stream_view(cudf::default_stream_value.value())); state.exec(nvbench::exec_tag::sync, [&](nvbench::launch& launch) { // Allocate device-side temporary storage & run algorithm - cudf::io::json::detail::parse_nested_json(input, stream_view); + cudf::io::json::detail::parse_nested_json(input, cudf::default_stream_value); }); }