From 44390034e163dbf69def27984ae759ec6322449d Mon Sep 17 00:00:00 2001 From: Matt Pulver Date: Wed, 20 Sep 2023 12:51:40 -0400 Subject: [PATCH 01/72] Pre-allocate approx_median/quantile in each QueryMemoryInitializer. (#7519) Signed-off-by: Misiu Godfrey --- DataMgr/Allocators/FastAllocator.h | 96 ++++++++ .../Descriptors/ApproxQuantileDescriptor.h | 26 ++ .../Descriptors/QueryMemoryDescriptor.cpp | 5 + .../Descriptors/QueryMemoryDescriptor.h | 12 +- QueryEngine/Descriptors/RowSetMemoryOwner.h | 17 +- QueryEngine/Execute.cpp | 33 ++- QueryEngine/GroupByAndAggregate.cpp | 25 ++ QueryEngine/GroupByAndAggregate.h | 2 + QueryEngine/QueryMemoryInitializer.cpp | 145 ++++++----- QueryEngine/QueryMemoryInitializer.h | 2 +- QueryEngine/RelAlgExecutionUnit.h | 15 ++ QueryEngine/ResultSetReductionJIT.cpp | 7 +- Shared/SimpleAllocator.h | 6 +- Shared/quantile.h | 45 +++- Tests/ExecuteTest.cpp | 231 +++++++++++++----- 15 files changed, 505 insertions(+), 162 deletions(-) create mode 100644 DataMgr/Allocators/FastAllocator.h create mode 100644 QueryEngine/Descriptors/ApproxQuantileDescriptor.h diff --git a/DataMgr/Allocators/FastAllocator.h b/DataMgr/Allocators/FastAllocator.h new file mode 100644 index 0000000000..d17fb4b178 --- /dev/null +++ b/DataMgr/Allocators/FastAllocator.h @@ -0,0 +1,96 @@ +/* + * Copyright 2023 HEAVY.AI, Inc. + * + * 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. + */ + +/** + * @file FastAllocator.h + * @brief Quickly allocate many memory pieces by reserving them ahead of time. + * Calls to allocate() are thread-safe. + */ + +#pragma once + +#include "Shared/SimpleAllocator.h" + +#include +#include +#include +#include + +namespace heavyai { +namespace allocator { +namespace detail { + +inline std::runtime_error outOfMemoryError(size_t n, size_t remaining, size_t capacity) { + std::ostringstream oss; + oss << "allocate(" << n << ") called but only " << remaining << " out of " << capacity + << " available."; + return std::runtime_error(oss.str()); +} + +// FastAllocator accepts a pre-allocated buffer of given capacity and +// allocates sequential chunks, tracking the size_ starting at size_=0. +// If size_ exceeds capacity_ then throw an exception. +// There is no deallocate() function, nor is there a destructor. +template +class FastAllocator : public SimpleAllocator { + public: + FastAllocator() : buffer_(nullptr), capacity_(0), size_(0) {} + FastAllocator(T* buffer, size_t capacity) + : buffer_(buffer), capacity_(capacity), size_(0) {} + FastAllocator(FastAllocator const&) = delete; + FastAllocator(FastAllocator&&) = delete; + FastAllocator& operator=(FastAllocator const&) = delete; + FastAllocator& operator=(FastAllocator&& rhs) { + buffer_ = rhs.buffer_; + capacity_ = rhs.capacity_; + size_ = rhs.size_; + rhs.reset(); + return *this; + } + + // Allocate n>0 elements of type T. Caller responsible for proper data alignment. + T* allocate(size_t const n) { + CHECK(n); + std::lock_guard lock_guard(mutex_); + if (n <= available()) { + T* const ptr = buffer_ + size_; + size_ += n; + return ptr; + } + throw outOfMemoryError(n, available(), capacity_); + } + + size_t available() const { return capacity_ - size_; } // number of available elements + size_t capacity() const { return capacity_; } // number of reserved elements + + protected: + void reset() { + buffer_ = nullptr; + capacity_ = 0u; + size_ = 0u; + } + + T* buffer_; // Pointer to reserved buffer. + size_t capacity_; // Number of elements of type T reserved. + size_t size_; // Number of elements of type T allocated. + mutable std::mutex mutex_; +}; + +} // namespace detail +} // namespace allocator +} // namespace heavyai + +using heavyai::allocator::detail::FastAllocator; diff --git a/QueryEngine/Descriptors/ApproxQuantileDescriptor.h b/QueryEngine/Descriptors/ApproxQuantileDescriptor.h new file mode 100644 index 0000000000..37fbe97564 --- /dev/null +++ b/QueryEngine/Descriptors/ApproxQuantileDescriptor.h @@ -0,0 +1,26 @@ +/* + * Copyright 2023 HEAVY.AI, Inc. + * + * 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 + +struct ApproxQuantileDescriptor { + size_t buffer_size; // number of elements in TDigest buffer + size_t centroids_size; // number of elements in TDigest centroids +}; + +using ApproxQuantileDescriptors = std::vector; diff --git a/QueryEngine/Descriptors/QueryMemoryDescriptor.cpp b/QueryEngine/Descriptors/QueryMemoryDescriptor.cpp index b58dede072..ac18594e6c 100644 --- a/QueryEngine/Descriptors/QueryMemoryDescriptor.cpp +++ b/QueryEngine/Descriptors/QueryMemoryDescriptor.cpp @@ -250,6 +250,7 @@ std::unique_ptr QueryMemoryDescriptor::init( const size_t shard_count, const size_t max_groups_buffer_entry_count, RenderInfo* render_info, + const ApproxQuantileDescriptors& approx_quantile_descriptors, const CountDistinctDescriptors count_distinct_descriptors, const bool must_use_baseline_sort, const bool output_columnar_hint, @@ -288,6 +289,7 @@ std::unique_ptr QueryMemoryDescriptor::init( /*group_col_compact_width=*/0, std::vector{}, /*entry_count=*/1, + approx_quantile_descriptors, count_distinct_descriptors, false, output_columnar_hint, @@ -428,6 +430,7 @@ std::unique_ptr QueryMemoryDescriptor::init( group_col_compact_width, target_groupby_indices, entry_count, + approx_quantile_descriptors, count_distinct_descriptors, sort_on_gpu_hint, output_columnar, @@ -461,6 +464,7 @@ QueryMemoryDescriptor::QueryMemoryDescriptor( const int8_t group_col_compact_width, const std::vector& target_groupby_indices, const size_t entry_count, + const ApproxQuantileDescriptors& approx_quantile_descriptors, const CountDistinctDescriptors count_distinct_descriptors, const bool sort_on_gpu_hint, const bool output_columnar_hint, @@ -482,6 +486,7 @@ QueryMemoryDescriptor::QueryMemoryDescriptor( , max_val_(col_range_info.max) , bucket_(col_range_info.bucket) , has_nulls_(col_range_info.has_nulls) + , approx_quantile_descriptors_(approx_quantile_descriptors) , count_distinct_descriptors_(count_distinct_descriptors) , output_columnar_(false) , render_output_(render_output) diff --git a/QueryEngine/Descriptors/QueryMemoryDescriptor.h b/QueryEngine/Descriptors/QueryMemoryDescriptor.h index b5db1099de..fd21c3c281 100644 --- a/QueryEngine/Descriptors/QueryMemoryDescriptor.h +++ b/QueryEngine/Descriptors/QueryMemoryDescriptor.h @@ -25,6 +25,7 @@ #include "../CompilationOptions.h" #include "../CountDistinct.h" +#include "ApproxQuantileDescriptor.h" #include "ColSlotContext.h" #include "Types.h" @@ -81,7 +82,8 @@ class QueryMemoryDescriptor { const int8_t group_col_compact_width, const std::vector& target_groupby_indices, const size_t entry_count, - const CountDistinctDescriptors count_distinct_descriptors, + const ApproxQuantileDescriptors&, + const CountDistinctDescriptors, const bool sort_on_gpu_hint, const bool output_columnar, const bool render_output, @@ -118,7 +120,8 @@ class QueryMemoryDescriptor { const size_t shard_count, const size_t max_groups_buffer_entry_count, RenderInfo* render_info, - const CountDistinctDescriptors count_distinct_descriptors, + const ApproxQuantileDescriptors&, + const CountDistinctDescriptors, const bool must_use_baseline_sort, const bool output_columnar_hint, const bool streaming_top_n_hint, @@ -263,6 +266,10 @@ class QueryMemoryDescriptor { bool hasNulls() const { return has_nulls_; } + const ApproxQuantileDescriptors& getApproxQuantileDescriptors() const { + return approx_quantile_descriptors_; + } + const CountDistinctDescriptor& getCountDistinctDescriptor(const size_t idx) const { CHECK_LT(idx, count_distinct_descriptors_.size()); return count_distinct_descriptors_[idx]; @@ -398,6 +405,7 @@ class QueryMemoryDescriptor { int64_t max_val_; int64_t bucket_; bool has_nulls_; + ApproxQuantileDescriptors approx_quantile_descriptors_; CountDistinctDescriptors count_distinct_descriptors_; bool sort_on_gpu_; bool output_columnar_; diff --git a/QueryEngine/Descriptors/RowSetMemoryOwner.h b/QueryEngine/Descriptors/RowSetMemoryOwner.h index b8f8fd445f..f7522e2eb9 100644 --- a/QueryEngine/Descriptors/RowSetMemoryOwner.h +++ b/QueryEngine/Descriptors/RowSetMemoryOwner.h @@ -17,6 +17,7 @@ #pragma once #include +#include #include #include #include @@ -24,8 +25,10 @@ #include #include +#include "ApproxQuantileDescriptor.h" #include "DataMgr/AbstractBuffer.h" #include "DataMgr/Allocators/ArenaAllocator.h" +#include "DataMgr/Allocators/FastAllocator.h" #include "DataMgr/DataMgr.h" #include "Logger/Logger.h" #include "QueryEngine/AggMode.h" @@ -66,7 +69,12 @@ class RowSetMemoryOwner final : public SimpleAllocator, boost::noncopyable { enum class StringTranslationType { SOURCE_INTERSECTION, SOURCE_UNION }; - int8_t* allocate(const size_t num_bytes, const size_t thread_idx = 0) override { + int8_t* allocate(const size_t num_bytes) override { + constexpr size_t thread_idx = 0u; + return allocate(num_bytes, thread_idx); + } + + int8_t* allocate(const size_t num_bytes, const size_t thread_idx) { CHECK_LT(thread_idx, allocators_.size()); auto allocator = allocators_[thread_idx].get(); std::lock_guard lock(state_mutex_); @@ -335,7 +343,8 @@ class RowSetMemoryOwner final : public SimpleAllocator, boost::noncopyable { return string_dictionary_generations_; } - quantile::TDigest* nullTDigest(double const q); + quantile::TDigest* initTDigest(size_t thread_idx, ApproxQuantileDescriptor, double q); + void reserveTDigestMemory(size_t thread_idx, size_t capacity); // // key/value store for table function intercommunication @@ -396,7 +405,11 @@ class RowSetMemoryOwner final : public SimpleAllocator, boost::noncopyable { StringDictionaryGenerations string_dictionary_generations_; std::vector col_buffers_; std::vector varlen_input_buffers_; + + using TDigestAllocator = FastAllocator; + std::deque t_digest_allocators_; std::vector> t_digests_; + std::map> string_ops_owned_; std::list mode_maps_; diff --git a/QueryEngine/Execute.cpp b/QueryEngine/Execute.cpp index d1c3712537..b83d4b533f 100644 --- a/QueryEngine/Execute.cpp +++ b/QueryEngine/Execute.cpp @@ -26,12 +26,12 @@ #include #include #include -#include #include #include #include #include #include +#include #include "Catalog/Catalog.h" #include "CudaMgr/CudaMgr.h" @@ -670,12 +670,33 @@ RowSetMemoryOwner::getOrAddStringProxyNumericTranslationMap( return addStringProxyNumericTranslationMap(source_proxy, string_op_infos); } -quantile::TDigest* RowSetMemoryOwner::nullTDigest(double const q) { +quantile::TDigest* RowSetMemoryOwner::initTDigest(size_t const thread_idx, + ApproxQuantileDescriptor const desc, + double const q) { + static_assert(std::is_trivially_copyable_v); std::lock_guard lock(state_mutex_); - return t_digests_ - .emplace_back(std::make_unique( - q, this, g_approx_quantile_buffer, g_approx_quantile_centroids)) - .get(); + auto t_digest = std::make_unique( + q, &t_digest_allocators_[thread_idx], desc.buffer_size, desc.centroids_size); + return t_digests_.emplace_back(std::move(t_digest)).get(); +} + +void RowSetMemoryOwner::reserveTDigestMemory(size_t thread_idx, size_t capacity) { + std::unique_lock lock(state_mutex_); + if (t_digest_allocators_.size() <= thread_idx) { + t_digest_allocators_.resize(thread_idx + 1u); + } + if (t_digest_allocators_[thread_idx].capacity()) { + // This can only happen when a thread_idx is re-used. In other words, + // two or more kernels have launched (serially!) using the same thread_idx. + // This is ok since TDigestAllocator does not own the memory it allocates. + VLOG(2) << "Replacing t_digest_allocators_[" << thread_idx << "]."; + } + lock.unlock(); + // This is not locked due to use of same state_mutex_ during allocation. + // The corresponding deallocation happens in ~DramArena(). + int8_t* const buffer = allocate(capacity, thread_idx); + lock.lock(); + t_digest_allocators_[thread_idx] = TDigestAllocator(buffer, capacity); } bool Executor::isCPUOnly() const { diff --git a/QueryEngine/GroupByAndAggregate.cpp b/QueryEngine/GroupByAndAggregate.cpp index 1483a25ae0..dcdd033f13 100644 --- a/QueryEngine/GroupByAndAggregate.cpp +++ b/QueryEngine/GroupByAndAggregate.cpp @@ -54,7 +54,10 @@ bool g_cluster{false}; bool g_bigint_count{false}; int g_hll_precision_bits{11}; size_t g_watchdog_baseline_max_groups{120000000}; +extern size_t g_approx_quantile_buffer; +extern size_t g_approx_quantile_centroids; extern int64_t g_bitmap_memory_limit; +extern size_t g_default_max_groups_buffer_entry_guess; extern size_t g_leaf_count; bool ColRangeInfo::isEmpty() const { @@ -888,6 +891,25 @@ std::unique_ptr GroupByAndAggregate::initQueryMemoryDescr return query_mem_desc; } +ApproxQuantileDescriptors GroupByAndAggregate::initApproxQuantileDescriptors() { + // Count APPROX_QUANTILE targets + size_t target_count = 0u; + auto count_target = [&](Analyzer::AggExpr const*, size_t) { ++target_count; }; + ra_exe_unit_.eachAggTarget(count_target); + if (target_count == 0u) { + return {}; + } + + // Reserve and fill descriptors + std::vector descriptors; + descriptors.reserve(target_count); + auto add_descriptor = [&](Analyzer::AggExpr const*, size_t) { + descriptors.push_back({g_approx_quantile_buffer, g_approx_quantile_centroids}); + }; + ra_exe_unit_.eachAggTarget(add_descriptor); + return descriptors; +} + std::unique_ptr GroupByAndAggregate::initQueryMemoryDescriptorImpl( const bool allow_multifrag, const size_t max_groups_buffer_entry_count, @@ -936,6 +958,7 @@ std::unique_ptr GroupByAndAggregate::initQueryMemoryDescr const auto count_distinct_descriptors = init_count_distinct_descriptors( ra_exe_unit_, query_infos_, col_range_info, device_type_, executor_); + auto approx_quantile_descriptors = initApproxQuantileDescriptors(); try { return QueryMemoryDescriptor::init(executor_, ra_exe_unit_, @@ -949,6 +972,7 @@ std::unique_ptr GroupByAndAggregate::initQueryMemoryDescr shard_count, max_groups_buffer_entry_count, render_info, + approx_quantile_descriptors, count_distinct_descriptors, must_use_baseline_sort, output_columnar_hint, @@ -968,6 +992,7 @@ std::unique_ptr GroupByAndAggregate::initQueryMemoryDescr shard_count, max_groups_buffer_entry_count, render_info, + approx_quantile_descriptors, count_distinct_descriptors, must_use_baseline_sort, output_columnar_hint, diff --git a/QueryEngine/GroupByAndAggregate.h b/QueryEngine/GroupByAndAggregate.h index 6c959be3a7..9ceb96bd26 100644 --- a/QueryEngine/GroupByAndAggregate.h +++ b/QueryEngine/GroupByAndAggregate.h @@ -80,6 +80,8 @@ class GroupByAndAggregate { private: bool gpuCanHandleOrderEntries(const std::list& order_entries); + ApproxQuantileDescriptors initApproxQuantileDescriptors(); + std::unique_ptr initQueryMemoryDescriptor( const bool allow_multifrag, const size_t max_groups_buffer_entry_count, diff --git a/QueryEngine/QueryMemoryInitializer.cpp b/QueryEngine/QueryMemoryInitializer.cpp index 62272cdf81..4799cc41c8 100644 --- a/QueryEngine/QueryMemoryInitializer.cpp +++ b/QueryEngine/QueryMemoryInitializer.cpp @@ -29,6 +29,14 @@ int64_t g_bitmap_memory_limit{8LL * 1000 * 1000 * 1000}; namespace { +struct AddNbytes { + size_t const entry_count; + size_t operator()(size_t const sum, ApproxQuantileDescriptor const aqd) const { + return sum + + entry_count * quantile::TDigest::nbytes(aqd.buffer_size, aqd.centroids_size); + } +}; + inline void check_total_bitmap_memory(const QueryMemoryDescriptor& query_mem_desc) { const size_t groups_buffer_entry_count = query_mem_desc.getEntryCount(); checked_int64_t total_bytes_per_group = 0; @@ -264,6 +272,18 @@ QueryMemoryInitializer::QueryMemoryInitializer( } } + if (agg_op_metadata.has_tdigest) { + auto const& descs = query_mem_desc.getApproxQuantileDescriptors(); + // Pre-allocate all TDigest memory for this thread. + AddNbytes const add_nbytes{query_mem_desc.getEntryCount()}; + size_t const capacity = + std::accumulate(descs.begin(), descs.end(), size_t(0), add_nbytes); + VLOG(2) << "row_set_mem_owner_->reserveTDigestMemory(" << thread_idx_ << ',' + << capacity << ") query_mem_desc.getEntryCount()(" + << query_mem_desc.getEntryCount() << ')'; + row_set_mem_owner_->reserveTDigestMemory(thread_idx_, capacity); + } + if (render_allocator_map || !query_mem_desc.isGroupBy()) { if (agg_op_metadata.has_count_distinct) { allocateCountDistinctBuffers(query_mem_desc, ra_exe_unit); @@ -289,7 +309,7 @@ QueryMemoryInitializer::QueryMemoryInitializer( initializeModeIndexSet(query_mem_desc, ra_exe_unit); } if (agg_op_metadata.has_tdigest) { - agg_op_metadata.qualtile_params = + agg_op_metadata.quantile_params = initializeQuantileParams(query_mem_desc, ra_exe_unit); } } @@ -320,6 +340,7 @@ QueryMemoryInitializer::QueryMemoryInitializer( const auto group_buffers_count = !query_mem_desc.isGroupBy() ? 1 : num_buffers_; int64_t* group_by_buffer_template{nullptr}; + if (!query_mem_desc.lazyInitGroups(device_type) && group_buffers_count > 1) { group_by_buffer_template = reinterpret_cast( row_set_mem_owner_->allocate(group_buffer_size, thread_idx_)); @@ -593,9 +614,9 @@ void QueryMemoryInitializer::initRowGroups(const QueryMemoryDescriptor& query_me auto buffer_ptr = reinterpret_cast(groups_buffer); const auto query_mem_desc_fixedup = ResultSet::fixupQueryMemoryDescriptor(query_mem_desc); + auto const key_sz = query_mem_desc.getEffectiveKeyWidth(); // not COUNT DISTINCT / APPROX_COUNT_DISTINCT / APPROX_QUANTILE // we use the default implementation in those agg ops - auto const key_sz = query_mem_desc.getEffectiveKeyWidth(); if (!(agg_op_metadata.has_count_distinct || agg_op_metadata.has_mode || agg_op_metadata.has_tdigest) && g_optimize_row_initialization) { @@ -747,6 +768,7 @@ void QueryMemoryInitializer::initColumnsPerRow( const TargetAggOpsMetadata& agg_op_metadata) { int8_t* col_ptr = row_ptr; size_t init_vec_idx = 0; + size_t approx_quantile_descriptors_idx = 0; for (size_t col_idx = 0; col_idx < query_mem_desc.getSlotCount(); col_ptr += query_mem_desc.getNextColOffInBytesRowOnly(col_ptr, col_idx++)) { int64_t init_val{0}; @@ -764,10 +786,12 @@ void QueryMemoryInitializer::initColumnsPerRow( ++init_vec_idx; } } else if (agg_op_metadata.has_tdigest && - agg_op_metadata.qualtile_params[col_idx]) { - auto const q = *agg_op_metadata.qualtile_params[col_idx]; - // allocate for APPROX_QUANTILE only when slot is used - init_val = reinterpret_cast(row_set_mem_owner_->nullTDigest(q)); + agg_op_metadata.quantile_params[col_idx]) { + auto const q = *agg_op_metadata.quantile_params[col_idx]; + auto const& descs = query_mem_desc.getApproxQuantileDescriptors(); + auto const& desc = descs.at(approx_quantile_descriptors_idx++); + init_val = reinterpret_cast( + row_set_mem_owner_->initTDigest(thread_idx_, desc, q)); CHECK_NE(init_val, 0); ++init_vec_idx; } else if (agg_op_metadata.has_mode && @@ -899,39 +923,18 @@ int64_t QueryMemoryInitializer::allocateCountDistinctSet() { return reinterpret_cast(count_distinct_set); } -namespace { - -void eachAggregateTargetIdxOfType( - std::vector const& target_exprs, - SQLAgg const agg_type, - std::function lambda) { - for (size_t target_idx = 0; target_idx < target_exprs.size(); ++target_idx) { - auto const target_expr = target_exprs[target_idx]; - if (auto const* agg_expr = dynamic_cast(target_expr)) { - if (agg_expr->get_aggtype() == agg_type) { - lambda(agg_expr, target_idx); - } - } - } -} - -} // namespace - QueryMemoryInitializer::ModeIndexSet QueryMemoryInitializer::initializeModeIndexSet( const QueryMemoryDescriptor& query_mem_desc, const RelAlgExecutionUnit& ra_exe_unit) { size_t const slot_count = query_mem_desc.getSlotCount(); CHECK_LE(ra_exe_unit.target_exprs.size(), slot_count); ModeIndexSet mode_index_set; - eachAggregateTargetIdxOfType( - ra_exe_unit.target_exprs, - kMODE, - [&](Analyzer::AggExpr const*, size_t const target_idx) { - size_t const agg_col_idx = - query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); - CHECK_LT(agg_col_idx, slot_count); - mode_index_set.emplace(agg_col_idx); - }); + ra_exe_unit.eachAggTarget([&](Analyzer::AggExpr const*, + size_t const target_idx) { + size_t const agg_col_idx = query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); + CHECK_LT(agg_col_idx, slot_count); + mode_index_set.emplace(agg_col_idx); + }); return mode_index_set; } @@ -940,16 +943,13 @@ void QueryMemoryInitializer::allocateModeBuffer( const RelAlgExecutionUnit& ra_exe_unit) { size_t const slot_count = query_mem_desc.getSlotCount(); CHECK_LE(ra_exe_unit.target_exprs.size(), slot_count); - eachAggregateTargetIdxOfType( - ra_exe_unit.target_exprs, - kMODE, - [&](Analyzer::AggExpr const*, size_t const target_idx) { - size_t const agg_col_idx = - query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); - CHECK_LT(agg_col_idx, slot_count); - AggMode* agg_mode = row_set_mem_owner_->allocateMode(); - init_agg_vals_[agg_col_idx] = reinterpret_cast(agg_mode); - }); + ra_exe_unit.eachAggTarget([&](Analyzer::AggExpr const*, + size_t const target_idx) { + size_t const agg_col_idx = query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); + CHECK_LT(agg_col_idx, slot_count); + AggMode* agg_mode = row_set_mem_owner_->allocateMode(); + init_agg_vals_[agg_col_idx] = reinterpret_cast(agg_mode); + }); } std::vector @@ -959,20 +959,17 @@ QueryMemoryInitializer::initializeQuantileParams( size_t const slot_count = query_mem_desc.getSlotCount(); CHECK_LE(ra_exe_unit.target_exprs.size(), slot_count); std::vector quantile_params(slot_count); - eachAggregateTargetIdxOfType( - ra_exe_unit.target_exprs, - kAPPROX_QUANTILE, - [&](Analyzer::AggExpr const* const agg_expr, size_t const target_idx) { - size_t const agg_col_idx = - query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); - CHECK_LT(agg_col_idx, slot_count); - CHECK_EQ(static_cast(sizeof(int64_t)), - query_mem_desc.getLogicalSlotWidthBytes(agg_col_idx)); - auto const q_expr = - dynamic_cast(agg_expr->get_arg1().get()); - CHECK(q_expr); - quantile_params[agg_col_idx] = q_expr->get_constval().doubleval; - }); + ra_exe_unit.eachAggTarget([&](Analyzer::AggExpr const* const agg_expr, + size_t const target_idx) { + size_t const agg_col_idx = query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); + CHECK_LT(agg_col_idx, slot_count); + CHECK_EQ(static_cast(sizeof(int64_t)), + query_mem_desc.getLogicalSlotWidthBytes(agg_col_idx)); + auto const q_expr = + dynamic_cast(agg_expr->get_arg1().get()); + CHECK(q_expr); + quantile_params[agg_col_idx] = q_expr->get_constval().doubleval; + }); return quantile_params; } @@ -981,23 +978,23 @@ void QueryMemoryInitializer::allocateTDigestsBuffer( const RelAlgExecutionUnit& ra_exe_unit) { size_t const slot_count = query_mem_desc.getSlotCount(); CHECK_LE(ra_exe_unit.target_exprs.size(), slot_count); - eachAggregateTargetIdxOfType( - ra_exe_unit.target_exprs, - kAPPROX_QUANTILE, - [&](Analyzer::AggExpr const* const agg_expr, size_t const target_idx) { - size_t const agg_col_idx = - query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); - CHECK_LT(agg_col_idx, slot_count); - CHECK_EQ(static_cast(sizeof(int64_t)), - query_mem_desc.getLogicalSlotWidthBytes(agg_col_idx)); - auto const q_expr = - dynamic_cast(agg_expr->get_arg1().get()); - CHECK(q_expr); - auto const q = q_expr->get_constval().doubleval; - // allocate for APPROX_QUANTILE only when slot is used - init_agg_vals_[agg_col_idx] = - reinterpret_cast(row_set_mem_owner_->nullTDigest(q)); - }); + + auto const& descs = query_mem_desc.getApproxQuantileDescriptors(); + size_t approx_quantile_descriptors_idx = 0u; + ra_exe_unit.eachAggTarget([&](Analyzer::AggExpr const* const agg_expr, + size_t const target_idx) { + size_t const agg_col_idx = query_mem_desc.getSlotIndexForSingleSlotCol(target_idx); + CHECK_LT(agg_col_idx, slot_count); + CHECK_EQ(static_cast(sizeof(int64_t)), + query_mem_desc.getLogicalSlotWidthBytes(agg_col_idx)); + auto const q_expr = + dynamic_cast(agg_expr->get_arg1().get()); + CHECK(q_expr); + auto const q = q_expr->get_constval().doubleval; + auto const& desc = descs.at(approx_quantile_descriptors_idx++); + init_agg_vals_[agg_col_idx] = + reinterpret_cast(row_set_mem_owner_->initTDigest(thread_idx_, desc, q)); + }); } GpuGroupByBuffers QueryMemoryInitializer::prepareTopNHeapsDevBuffer( diff --git a/QueryEngine/QueryMemoryInitializer.h b/QueryEngine/QueryMemoryInitializer.h index f193a58241..e4e300a887 100644 --- a/QueryEngine/QueryMemoryInitializer.h +++ b/QueryEngine/QueryMemoryInitializer.h @@ -42,7 +42,7 @@ class QueryMemoryInitializer { bool has_tdigest{false}; std::vector count_distinct_buf_size; ModeIndexSet mode_index_set; - std::vector qualtile_params; + std::vector quantile_params; }; // Row-based execution constructor diff --git a/QueryEngine/RelAlgExecutionUnit.h b/QueryEngine/RelAlgExecutionUnit.h index 72d55bf82b..1764116881 100644 --- a/QueryEngine/RelAlgExecutionUnit.h +++ b/QueryEngine/RelAlgExecutionUnit.h @@ -35,6 +35,7 @@ #include +#include #include #include #include @@ -187,6 +188,20 @@ struct RelAlgExecutionUnit { RelAlgExecutionUnit createNdvExecutionUnit(const int64_t range) const; RelAlgExecutionUnit createCountAllExecutionUnit( Analyzer::Expr* replacement_target) const; + + // Call lambda() for each aggregate target_expr of SQLAgg type AggType. + template + void eachAggTarget( + std::function lambda) const { + for (size_t target_idx = 0; target_idx < target_exprs.size(); ++target_idx) { + Analyzer::Expr const* target_expr = target_exprs[target_idx]; + if (auto const* agg_expr = dynamic_cast(target_expr)) { + if (agg_expr->get_aggtype() == AggType) { + lambda(agg_expr, target_idx); + } + } + } + } }; std::ostream& operator<<(std::ostream& os, const RelAlgExecutionUnit& ra_exe_unit); diff --git a/QueryEngine/ResultSetReductionJIT.cpp b/QueryEngine/ResultSetReductionJIT.cpp index 3f030869b9..2219e2c1aa 100644 --- a/QueryEngine/ResultSetReductionJIT.cpp +++ b/QueryEngine/ResultSetReductionJIT.cpp @@ -463,8 +463,11 @@ extern "C" RUNTIME_EXPORT void approx_quantile_jit_rt(const int64_t new_set_hand auto* incoming = reinterpret_cast(new_set_handle); if (incoming->centroids().capacity()) { auto* accumulator = reinterpret_cast(old_set_handle); - accumulator->allocate(); - accumulator->mergeTDigest(*incoming); + if (accumulator->centroids().capacity() == 0u) { + *accumulator = std::move(*incoming); + } else { + accumulator->mergeTDigest(*incoming); + } } } diff --git a/Shared/SimpleAllocator.h b/Shared/SimpleAllocator.h index 9d0a6acb9b..cf61c1519a 100644 --- a/Shared/SimpleAllocator.h +++ b/Shared/SimpleAllocator.h @@ -17,9 +17,7 @@ #pragma once class SimpleAllocator { - protected: - ~SimpleAllocator() = default; - public: - virtual int8_t* allocate(const size_t num_bytes, const size_t thread_idx = 0) = 0; + virtual ~SimpleAllocator() = default; + virtual int8_t* allocate(const size_t num_bytes) = 0; }; diff --git a/Shared/quantile.h b/Shared/quantile.h index 20eb1aa5e6..0022572f6c 100644 --- a/Shared/quantile.h +++ b/Shared/quantile.h @@ -189,15 +189,16 @@ class TDigest { Centroids centroids_; bool forward_{true}; // alternate direction on each call to mergeCentroids(). #ifndef __CUDACC__ - std::once_flag merge_buffer_final_once_; + std::mutex merge_buffer_final_called_mutex_; #endif + bool merge_buffer_final_called_{false}; // simple_allocator_, buf_allocate_, centroids_allocate_ are used only by allocate(). - std::optional const q_{std::nullopt}; // Optional preset quantile parameter. - bool const use_linear_scaling_function_{false}; - SimpleAllocator* const simple_allocator_{nullptr}; - IndexType const buf_allocate_{0}; - IndexType const centroids_allocate_{0}; + std::optional q_{std::nullopt}; // Optional preset quantile parameter. + bool use_linear_scaling_function_{false}; + SimpleAllocator* simple_allocator_{nullptr}; + IndexType buf_allocate_{0}; + IndexType centroids_allocate_{0}; DEVICE RealType max() const { return centroids_.max_; @@ -242,6 +243,25 @@ class TDigest { , buf_allocate_(buf_allocate) , centroids_allocate_(centroids_allocate) {} + // Called by approx_quantile_jit_rt(). Move everything except the unmovable mutex. + TDigest& operator=(TDigest&& rhs) { + buf_ = std::move(rhs.buf_); + centroids_ = std::move(rhs.centroids_); + forward_ = std::move(rhs.forward_); + merge_buffer_final_called_ = std::move(rhs.merge_buffer_final_called_); + q_ = std::move(rhs.q_); + use_linear_scaling_function_ = std::move(rhs.use_linear_scaling_function_); + simple_allocator_ = std::move(rhs.simple_allocator_); + buf_allocate_ = std::move(rhs.buf_allocate_); + centroids_allocate_ = std::move(rhs.centroids_allocate_); + return *this; + } + + // Size of reserved buffer+centroids space used for each TDigest in bytes. + static IndexType nbytes(IndexType buf_allocate, IndexType centroids_allocate) { + return (buf_allocate + centroids_allocate) * (sizeof(RealType) + sizeof(IndexType)); + } + DEVICE Centroids& centroids() { return centroids_; } @@ -661,16 +681,15 @@ DEVICE void TDigest::mergeBuffer() { // [QE-383] Make concurrent calls to mergeBufferFinal() thread-safe. template DEVICE void TDigest::mergeBufferFinal() { - auto const call_once = [this] { +#ifndef __CUDACC__ + std::lock_guard lock_guard(merge_buffer_final_called_mutex_); +#endif + if (!merge_buffer_final_called_) { mergeBuffer(); assert(centroids_.size() <= buf_.capacity()); partialSumOfCounts(buf_.counts_.data()); - }; -#ifdef __CUDACC__ - call_once(); -#else - std::call_once(merge_buffer_final_once_, call_once); -#endif + merge_buffer_final_called_ = true; + } } template diff --git a/Tests/ExecuteTest.cpp b/Tests/ExecuteTest.cpp index a011277eed..5fabb1f51a 100644 --- a/Tests/ExecuteTest.cpp +++ b/Tests/ExecuteTest.cpp @@ -103,6 +103,83 @@ size_t choose_shard_count() { return g_num_leafs * (device_count > 1 ? device_count : 1); } +enum class ColumnDefinitionEnum : unsigned { + NAME = 0, // column name + HEAVY, // column definition as used in heavydb CREATE TABLE + SQLITE, // column definition as used in SQLite CREATE TABLE + N +}; + +// Indexed by ColumnDefinitionEnum +using ColumnDefinition = std::array; + +struct TableDefinition { + std::vector column_definitions; + // Add metadata as needed + + // Return column definitions for e = HEAVY or SQLITE. + std::string columnDefinitions(ColumnDefinitionEnum const e) const { + std::ostringstream oss; + for (size_t i = 0; i < column_definitions.size(); ++i) { + oss << (i ? ", " : "") + << column_definitions[i][unsigned(ColumnDefinitionEnum::NAME)] << ' ' + << column_definitions[i][unsigned(e)]; + } + return oss.str(); + } +}; + +// clang-format off +// table name -> TableDefinition +std::map const g_table_definitions = +{ + { "test", TableDefinition + { // NAME, HEAVY type, SQLITE type + { { "x", "int not null", "int not null" } + , { "w", "tinyint", "tinyint" } + , { "y", "int", "int" } + , { "z", "smallint", "smallint" } + , { "t", "bigint", "bigint" } + , { "b", "boolean", "boolean" } + , { "f", "float", "float" } + , { "ff", "float", "float" } + , { "fn", "float", "float" } + , { "d", "double", "double" } + , { "dn", "double", "double" } + , { "str", "varchar(10)", "varchar(10)" } + , { "null_str", "text encoding dict", "text" } + , { "fixed_str", "text encoding dict(16)", "text" } + , { "fixed_null_str", "text encoding dict(16)", "text" } + , { "real_str", "text encoding none", "text" } + , { "shared_dict", "text", "text" } + , { "m", "timestamp(0)", "timestamp(0)" } + , { "me", "timestamp(0) encoding fixed(32)", "timestamp(0)" } + , { "m_3", "timestamp(3)", "timestamp(3)" } + , { "m_6", "timestamp(6)", "timestamp(6)" } + , { "m_9", "timestamp(9)", "timestamp(9)" } + , { "n", "time(0)", "time(0)" } + , { "ne", "time encoding fixed(32)", "time(0)" } + , { "o", "date", "date" } + , { "o1", "date encoding fixed(16)", "date" } + , { "o2", "date encoding fixed(32)", "date" } + , { "fx", "int encoding fixed(16)", "int" } + , { "dd", "decimal(10, 2)", "decimal(10, 2)" } + , { "dd_notnull", "decimal(10, 2) not null", "decimal(10, 2) not null" } + , { "ss", "text encoding dict", "text" } + , { "u", "int", "int" } + , { "ofd", "int", "int" } + , { "ufd", "int not null", "int not null" } + , { "ofq", "bigint", "bigint" } + , { "ufq", "bigint not null", "bigint not null" } + , { "smallint_nulls", "smallint", "smallint" } + , { "bn", "boolean not null", "boolean not null" } + , { "num_text", "text encoding dict", "text" } + } + } + } +}; +// clang-format on + std::shared_ptr run_multiple_agg(const string& query_str, const ExecutorDeviceType device_type, const bool allow_loop_joins) { @@ -3932,16 +4009,73 @@ INSTANTIATE_TEST_SUITE_P( testing::Values(kAVG, kSUM, kAPPROX_QUANTILE, kSAMPLE, kSINGLE_VALUE, kMODE)), AggDistinctUnsupported::testName); -// Additional unit tests for APPROX_MEDIAN are in Quantile/. -TEST_F(Select, ApproxMedianSanity) { - auto dt = ExecutorDeviceType::CPU; - auto approx_median = [dt](std::string const col) { - std::string const query = "SELECT APPROX_MEDIAN(" + col + ") FROM test;"; - return v(run_simple_agg(query, dt)); - }; +struct TestParam { + double expected; + char const* column; +}; + +using ApproxMedianParam = std::tuple; + +class ApproxMedian : public Select, + public testing::WithParamInterface { + public: + static void executeQueryAndAssertResult(ExecutorDeviceType const dt, + TestParam const params) { + auto query = "SELECT APPROX_MEDIAN(" + std::string(params.column) + ") FROM test;"; + double actual = v(run_simple_agg(query, dt)); + EXPECT_EQ(params.expected, actual); + } + + // Replace non-alphanumeric chars w/ underscore. Don't repeat underscores. + static std::string escape(std::string_view const sv) { + std::string escaped; + escaped.reserve(sv.size()); + bool last_was_underscore = false; + for (char const c : sv) { + if (std::isalnum(c)) { + escaped.push_back(c); + last_was_underscore = false; + } else { + if (!last_was_underscore) { + escaped.push_back('_'); + } + last_was_underscore = true; + } + } + if (last_was_underscore) { + escaped.pop_back(); + } + return escaped; + } + + // Return map of name -> column type for given table. + static std::map getColType(std::string_view table) { + std::map col_type; + for (ColumnDefinition const cd : g_table_definitions.at(table).column_definitions) { + col_type.emplace(cd[unsigned(ColumnDefinitionEnum::NAME)], + cd[unsigned(ColumnDefinitionEnum::HEAVY)]); + } + return col_type; + } + + // NOTE: test names must be non-empty, unique, and may only contain ASCII alphanumeric + // characters. In particular, they should not contain underscores* + // https://google.github.io/googletest/faq.html#why-should-test-suite-names-and-test-names-not-contain-underscore + static std::string testName(testing::TestParamInfo const& info) { + static std::map const col_type = + getColType("test"); + std::ostringstream oss; + char const* column_name = std::get<1>(info.param).column; + oss << std::get<0>(info.param) << '_' << column_name << '_' + << escape(col_type.at(column_name)); + return oss.str(); + } +}; + +TEST_P(ApproxMedian, AllColumnTypes) { if (g_aggregator) { try { - approx_median("w"); + executeQueryAndAssertResult(std::get<0>(GetParam()), std::get<1>(GetParam())); EXPECT_TRUE(false) << "Exception expected for approx_median query."; } catch (std::runtime_error const& e) { EXPECT_EQ(std::string(e.what()), @@ -3952,27 +4086,35 @@ TEST_F(Select, ApproxMedianSanity) { EXPECT_TRUE(false) << "std::runtime_error expected for approx_median query."; } } else { - EXPECT_EQ(-7.5, approx_median("w")); - EXPECT_EQ(7.0, approx_median("x")); - EXPECT_EQ(42.5, approx_median("y")); - EXPECT_EQ(101.0, approx_median("z")); - EXPECT_EQ(1001.5, approx_median("t")); - EXPECT_EQ((double(1.1f) + double(1.2f)) / 2, approx_median("f")); - EXPECT_EQ((double(1.1f) + double(101.2f)) / 2, approx_median("ff")); - EXPECT_EQ((double(-101.2f) + double(-1000.3f)) / 2, approx_median("fn")); - EXPECT_EQ(2.3, approx_median("d")); - EXPECT_EQ(-1111.5, approx_median("dn")); - EXPECT_EQ((11110.0 / 100 + 22220.0 / 100) / 2, approx_median("dd")); - EXPECT_EQ((11110.0 / 100 + 22220.0 / 100) / 2, approx_median("dd_notnull")); - EXPECT_EQ(NULL_DOUBLE, approx_median("u")); - EXPECT_EQ(2147483647.0, approx_median("ofd")); - EXPECT_EQ(-2147483647.5, approx_median("ufd")); - EXPECT_EQ(4611686018427387904.0, approx_median("ofq")); - EXPECT_EQ(-4611686018427387904.5, approx_median("ufq")); - EXPECT_EQ(32767.0, approx_median("smallint_nulls")); + executeQueryAndAssertResult(std::get<0>(GetParam()), std::get<1>(GetParam())); } } +INSTANTIATE_TEST_SUITE_P( + Select, // Just a name - not the class. This can be any symbol. + ApproxMedian, + testing::Combine(testing::Values(ExecutorDeviceType::CPU, ExecutorDeviceType::GPU), + testing::Values(TestParam{-7.5, "w"}, + TestParam{7.0, "x"}, + TestParam{42.5, "y"}, + TestParam{101.0, "z"}, + TestParam{1001.5, "t"}, + TestParam{(double(1.1f) + double(1.2f)) / 2, "f"}, + TestParam{(double(1.1f) + double(101.2f)) / 2, "ff"}, + TestParam{(double(-101.2f) + double(-1000.3f)) / 2, + "fn"}, + TestParam{2.3, "d"}, + TestParam{-1111.5, "dn"}, + TestParam{(111.1 + 222.2) / 2, "dd"}, + TestParam{(111.1 + 222.2) / 2, "dd_notnull"}, + TestParam{NULL_DOUBLE, "u"}, + TestParam{2147483647.0, "ofd"}, + TestParam{-2147483647.5, "ufd"}, + TestParam{4611686018427387904.0, "ofq"}, + TestParam{-4611686018427387904.5, "ufq"}, + TestParam{32767.0, "smallint_nulls"})), + ApproxMedian::testName); + TEST_F(Select, ApproxMedianLargeInts) { if (g_aggregator) { LOG(WARNING) << "Skipping ApproxMedianLargeInts tests in distributed mode."; @@ -28117,25 +28259,9 @@ int create_and_populate_tables(const bool use_temporary_tables, const std::string drop_old_test{"DROP TABLE IF EXISTS test;"}; run_ddl_statement(drop_old_test); g_sqlite_comparator.query(drop_old_test); - std::string columns_definition{ - "x int not null, w tinyint, y int, z smallint, t bigint, b boolean, f float, " - "ff " - "float, fn " - "float, d double, dn double, str " - "varchar(10), null_str text encoding dict, fixed_str text encoding dict(16), " - "fixed_null_str text encoding " - "dict(16), real_str text encoding none, shared_dict text, m timestamp(0), me " - "timestamp(0) encoding fixed(32), m_3 " - "timestamp(3), m_6 timestamp(6), " - "m_9 timestamp(9), n time(0), ne time encoding fixed(32), o date, o1 date " - "encoding fixed(16), o2 date " - "encoding fixed(32), fx int " - "encoding fixed(16), dd decimal(10, 2), dd_notnull decimal(10, 2) not null, ss " - "text encoding dict, u int, ofd " - "int, ufd int not null, ofq bigint, ufq bigint not null, smallint_nulls " - "smallint, bn boolean not null, num_text text encoding dict"}; + TableDefinition const& test_definition = g_table_definitions.at("test"); const std::string create_test = build_create_table_statement( - columns_definition, + test_definition.columnDefinitions(ColumnDefinitionEnum::HEAVY), "test", {g_shard_count ? "str" : "", g_shard_count}, {{"str", "test_inner", "str"}, {"shared_dict", "test", "str"}}, @@ -28144,20 +28270,9 @@ int create_and_populate_tables(const bool use_temporary_tables, with_delete_support); run_ddl_statement(create_test); g_sqlite_comparator.query( - "CREATE TABLE test(x int not null, w tinyint, y int, z smallint, t bigint, b " - "boolean, f " - "float, ff float, fn float, d " - "double, dn double, str varchar(10), null_str text, fixed_str text, " - "fixed_null_str text, real_str text, " - "shared_dict " - "text, m timestamp(0), me timestamp(0), m_3 timestamp(3), m_6 timestamp(6), " - "m_9 " - "timestamp(9), n " - "time(0), ne time(0), o date, o1 date, o2 date, " - "fx int, dd decimal(10, 2), dd_notnull decimal(10, 2) not " - "null, ss " - "text, u int, ofd int, ufd int not null, ofq bigint, ufq bigint not null, " - "smallint_nulls smallint, bn boolean not null, num_text text);"); + "CREATE TABLE test(" + + test_definition.columnDefinitions(ColumnDefinitionEnum::SQLITE) + ");"); + } catch (...) { LOG(ERROR) << "Failed to (re-)create table 'test'"; return -EEXIST; From d469c615565c65857d6131e0b95dbdd2a208724f Mon Sep 17 00:00:00 2001 From: Matt Pulver Date: Wed, 20 Sep 2023 14:00:11 -0400 Subject: [PATCH 02/72] Fix valgrind error Invalid write of size 1 in CudaMgr::loadGpuModuleData(). (#7521) Signed-off-by: Misiu Godfrey --- QueryEngine/NativeCodegen.cpp | 18 ++-- QueryEngine/NvidiaKernel.cpp | 171 +++++++++++++++++----------------- QueryEngine/NvidiaKernel.h | 9 ++ Tests/GpuSharedMemoryTest.cpp | 17 ++-- 4 files changed, 107 insertions(+), 108 deletions(-) diff --git a/QueryEngine/NativeCodegen.cpp b/QueryEngine/NativeCodegen.cpp index 2b37f573d8..76c4334653 100644 --- a/QueryEngine/NativeCodegen.cpp +++ b/QueryEngine/NativeCodegen.cpp @@ -1367,29 +1367,23 @@ std::shared_ptr CodeGenerator::generateNativeGPUCode( } LOG(PTX) << "PTX for the GPU:\n" << ptx << "\nEnd of PTX"; - auto cubin_result = ptx_to_cubin(ptx, gpu_target.cuda_mgr); - auto& option_keys = cubin_result.option_keys; - auto& option_values = cubin_result.option_values; - auto cubin = cubin_result.cubin; - auto link_state = cubin_result.link_state; - const auto num_options = option_keys.size(); - + CubinResult cubin_result = ptx_to_cubin(ptx, gpu_target.cuda_mgr); auto func_name = wrapper_func->getName().str(); auto gpu_compilation_context = std::make_shared(); for (int device_id = 0; device_id < gpu_target.cuda_mgr->getDeviceCount(); ++device_id) { gpu_compilation_context->addDeviceCode( - std::make_unique(cubin, + std::make_unique(cubin_result.cubin, cubin_result.cubin_size, func_name, device_id, gpu_target.cuda_mgr, - num_options, - &option_keys[0], - &option_values[0])); + cubin_result.option_keys.size(), + cubin_result.option_keys.data(), + cubin_result.option_values.data())); } - checkCudaErrors(cuLinkDestroy(link_state)); + checkCudaErrors(cuLinkDestroy(cubin_result.link_state)); return gpu_compilation_context; #else return {}; diff --git a/QueryEngine/NvidiaKernel.cpp b/QueryEngine/NvidiaKernel.cpp index 3983e03eb9..fc993bd88f 100644 --- a/QueryEngine/NvidiaKernel.cpp +++ b/QueryEngine/NvidiaKernel.cpp @@ -14,43 +14,48 @@ * limitations under the License. */ -#include - #include "NvidiaKernel.h" - #include "Logger/Logger.h" #include "OSDependent/heavyai_path.h" #include +#include + #ifdef HAVE_CUDA -namespace { -#define JIT_LOG_SIZE 8192 - -void fill_options(std::vector& option_keys, - std::vector& option_values, - char* info_log, - char* error_log) { - option_keys.push_back(CU_JIT_LOG_VERBOSE); - option_values.push_back(reinterpret_cast(1)); - option_keys.push_back(CU_JIT_THREADS_PER_BLOCK); - // fix the minimum # threads per block to the hardware-limit maximum num threads - // to avoid recompiling jit module even if we manipulate it via query hint - // (and allowed `CU_JIT_THREADS_PER_BLOCK` range is between 1 and 1024 by query hint) - option_values.push_back(reinterpret_cast(1024)); - option_keys.push_back(CU_JIT_WALL_TIME); - option_values.push_back(reinterpret_cast(0)); - option_keys.push_back(CU_JIT_INFO_LOG_BUFFER); - option_values.push_back(reinterpret_cast(info_log)); - option_keys.push_back(CU_JIT_INFO_LOG_BUFFER_SIZE_BYTES); - option_values.push_back(reinterpret_cast((long)JIT_LOG_SIZE)); - option_keys.push_back(CU_JIT_ERROR_LOG_BUFFER); - option_values.push_back(reinterpret_cast(error_log)); - option_keys.push_back(CU_JIT_ERROR_LOG_BUFFER_SIZE_BYTES); - option_values.push_back(reinterpret_cast((long)JIT_LOG_SIZE)); +CubinResult::CubinResult() + : cubin(nullptr), link_state(CUlinkState{}), cubin_size(0u), jit_wall_time_idx(0u) { + constexpr size_t JIT_LOG_SIZE = 8192u; + static_assert(0u < JIT_LOG_SIZE); + info_log.resize(JIT_LOG_SIZE - 1u); // minus 1 for null terminator + error_log.resize(JIT_LOG_SIZE - 1u); + std::pair options[] = { + {CU_JIT_LOG_VERBOSE, reinterpret_cast(1)}, + // fix the minimum # threads per block to the hardware-limit maximum num threads to + // avoid recompiling jit module even if we manipulate it via query hint (and allowed + // `CU_JIT_THREADS_PER_BLOCK` range is between 1 and 1024 by query hint) + {CU_JIT_THREADS_PER_BLOCK, reinterpret_cast(1024)}, + {CU_JIT_WALL_TIME, nullptr}, // input not read, only output + {CU_JIT_INFO_LOG_BUFFER, reinterpret_cast(&info_log[0])}, + {CU_JIT_INFO_LOG_BUFFER_SIZE_BYTES, reinterpret_cast(JIT_LOG_SIZE)}, + {CU_JIT_ERROR_LOG_BUFFER, reinterpret_cast(&error_log[0])}, + {CU_JIT_ERROR_LOG_BUFFER_SIZE_BYTES, reinterpret_cast(JIT_LOG_SIZE)}}; + constexpr size_t n_options = sizeof(options) / sizeof(*options); + option_keys.reserve(n_options); + option_values.reserve(n_options); + for (size_t i = 0; i < n_options; ++i) { + option_keys.push_back(options[i].first); + option_values.push_back(options[i].second); + if (options[i].first == CU_JIT_WALL_TIME) { + jit_wall_time_idx = i; + } + } + CHECK_EQ(CU_JIT_WALL_TIME, option_keys[jit_wall_time_idx]) << jit_wall_time_idx; } +namespace { + boost::filesystem::path get_gpu_rt_path() { boost::filesystem::path gpu_rt_path{heavyai::get_root_abs_path()}; gpu_rt_path /= "QueryEngine"; @@ -77,38 +82,38 @@ boost::filesystem::path get_cuda_table_functions_path() { } // namespace void nvidia_jit_warmup() { - std::vector option_keys; - std::vector option_values; - char info_log[JIT_LOG_SIZE]; - char error_log[JIT_LOG_SIZE]; - fill_options(option_keys, option_values, info_log, error_log); - CHECK_EQ(option_values.size(), option_keys.size()); - unsigned num_options = option_keys.size(); - CUlinkState link_state; - checkCudaErrors( - cuLinkCreate(num_options, &option_keys[0], &option_values[0], &link_state)) - << ": " << std::string(error_log); - VLOG(1) << "CUDA JIT time to create link: " - << *reinterpret_cast(&option_values[2]); + CubinResult cubin_result{}; + CHECK_EQ(cubin_result.option_values.size(), cubin_result.option_keys.size()); + unsigned const num_options = cubin_result.option_keys.size(); + checkCudaErrors(cuLinkCreate(num_options, + cubin_result.option_keys.data(), + cubin_result.option_values.data(), + &cubin_result.link_state)) + << ": " << cubin_result.error_log.c_str(); + VLOG(1) << "CUDA JIT time to create link: " << cubin_result.jitWallTime(); boost::filesystem::path gpu_rt_path = get_gpu_rt_path(); boost::filesystem::path cuda_table_functions_path = get_cuda_table_functions_path(); CHECK(!gpu_rt_path.empty()); CHECK(!cuda_table_functions_path.empty()); - checkCudaErrors(cuLinkAddFile( - link_state, CU_JIT_INPUT_FATBINARY, gpu_rt_path.c_str(), 0, nullptr, nullptr)) - << ": " << std::string(error_log); - VLOG(1) << "CUDA JIT time to add RT fatbinary: " - << *reinterpret_cast(&option_values[2]); - checkCudaErrors(cuLinkAddFile(link_state, + checkCudaErrors(cuLinkAddFile(cubin_result.link_state, + CU_JIT_INPUT_FATBINARY, + gpu_rt_path.c_str(), + 0, + nullptr, + nullptr)) + << ": " << cubin_result.error_log.c_str(); + VLOG(1) << "CUDA JIT time to add RT fatbinary: " << cubin_result.jitWallTime(); + checkCudaErrors(cuLinkAddFile(cubin_result.link_state, CU_JIT_INPUT_LIBRARY, cuda_table_functions_path.c_str(), 0, nullptr, nullptr)) - << ": " << std::string(error_log); + << ": " << cubin_result.error_log.c_str(); VLOG(1) << "CUDA JIT time to add GPU table functions library: " - << *reinterpret_cast(&option_values[2]); - checkCudaErrors(cuLinkDestroy(link_state)) << ": " << std::string(error_log); + << cubin_result.jitWallTime(); + checkCudaErrors(cuLinkDestroy(cubin_result.link_state)) + << ": " << cubin_result.error_log.c_str(); } std::string add_line_numbers(const std::string& text) { @@ -130,19 +135,14 @@ CubinResult ptx_to_cubin(const std::string& ptx, CHECK(!ptx.empty()); CHECK(cuda_mgr && cuda_mgr->getDeviceCount() > 0); cuda_mgr->setContext(0); - std::vector option_keys; - std::vector option_values; - char info_log[JIT_LOG_SIZE]; - char error_log[JIT_LOG_SIZE]; - fill_options(option_keys, option_values, info_log, error_log); - CHECK_EQ(option_values.size(), option_keys.size()); - unsigned num_options = option_keys.size(); - CUlinkState link_state; - checkCudaErrors( - cuLinkCreate(num_options, &option_keys[0], &option_values[0], &link_state)) - << ": " << std::string(error_log); - VLOG(1) << "CUDA JIT time to create link: " - << *reinterpret_cast(&option_values[2]); + CubinResult cubin_result{}; + CHECK_EQ(cubin_result.option_values.size(), cubin_result.option_keys.size()); + checkCudaErrors(cuLinkCreate(cubin_result.option_keys.size(), + cubin_result.option_keys.data(), + cubin_result.option_values.data(), + &cubin_result.link_state)) + << ": " << cubin_result.error_log.c_str(); + VLOG(1) << "CUDA JIT time to create link: " << cubin_result.jitWallTime(); boost::filesystem::path gpu_rt_path = get_gpu_rt_path(); boost::filesystem::path cuda_table_functions_path = get_cuda_table_functions_path(); @@ -152,21 +152,26 @@ CubinResult ptx_to_cubin(const std::string& ptx, // 1. nvcc -std=c++11 -arch=sm_35 --device-link -c [list of .cu files] // 2. nvcc -std=c++11 -arch=sm_35 -lib [list of .o files generated by step 1] -o // [library_name.a] - checkCudaErrors(cuLinkAddFile( - link_state, CU_JIT_INPUT_FATBINARY, gpu_rt_path.c_str(), 0, nullptr, nullptr)) - << ": " << std::string(error_log); - VLOG(1) << "CUDA JIT time to add RT fatbinary: " - << *reinterpret_cast(&option_values[2]); - checkCudaErrors(cuLinkAddFile(link_state, + checkCudaErrors(cuLinkAddFile(cubin_result.link_state, + CU_JIT_INPUT_FATBINARY, + gpu_rt_path.c_str(), + 0, + nullptr, + nullptr)) + << ": " << cubin_result.error_log.c_str(); + VLOG(1) << "CUDA JIT time to add RT fatbinary: " << cubin_result.jitWallTime(); + checkCudaErrors(cuLinkAddFile(cubin_result.link_state, CU_JIT_INPUT_LIBRARY, cuda_table_functions_path.c_str(), 0, nullptr, nullptr)) - << ": " << std::string(error_log); + << ": " << cubin_result.error_log.c_str(); VLOG(1) << "CUDA JIT time to add GPU table functions library: " - << *reinterpret_cast(&option_values[2]); - checkCudaErrors(cuLinkAddData(link_state, + << cubin_result.jitWallTime(); + // The ptx.length() + 1 follows the example in + // https://developer.nvidia.com/blog/discovering-new-features-in-cuda-11-4/ + checkCudaErrors(cuLinkAddData(cubin_result.link_state, CU_JIT_INPUT_PTX, static_cast(const_cast(ptx.c_str())), ptx.length() + 1, @@ -174,23 +179,19 @@ CubinResult ptx_to_cubin(const std::string& ptx, 0, nullptr, nullptr)) - << ": " << std::string(error_log) << "\nPTX:\n" + << ": " << cubin_result.error_log.c_str() << "\nPTX:\n" << add_line_numbers(ptx) << "\nEOF PTX"; - VLOG(1) << "CUDA JIT time to add generated code: " - << *reinterpret_cast(&option_values[2]); - void* cubin{nullptr}; - size_t cubinSize{0}; - checkCudaErrors(cuLinkComplete(link_state, &cubin, &cubinSize)) - << ": " << std::string(error_log); - VLOG(1) << "CUDA Linker completed: " << info_log; - CHECK(cubin); - CHECK_GT(cubinSize, size_t(0)); - VLOG(1) << "Generated GPU binary code size: " << cubinSize << " bytes"; - return {cubin, option_keys, option_values, link_state, cubinSize}; + VLOG(1) << "CUDA JIT time to add generated code: " << cubin_result.jitWallTime(); + checkCudaErrors(cuLinkComplete( + cubin_result.link_state, &cubin_result.cubin, &cubin_result.cubin_size)) + << ": " << cubin_result.error_log.c_str(); + VLOG(1) << "CUDA Linker completed: " << cubin_result.info_log.c_str(); + CHECK(cubin_result.cubin); + CHECK_LT(0u, cubin_result.cubin_size); + VLOG(1) << "Generated GPU binary code size: " << cubin_result.cubin_size << " bytes"; + return cubin_result; } -#endif -#ifdef HAVE_CUDA GpuDeviceCompilationContext::GpuDeviceCompilationContext(const void* image, const size_t module_size, const std::string& kernel_name, diff --git a/QueryEngine/NvidiaKernel.h b/QueryEngine/NvidiaKernel.h index 3eb68b9639..f47bac2dd7 100644 --- a/QueryEngine/NvidiaKernel.h +++ b/QueryEngine/NvidiaKernel.h @@ -33,6 +33,15 @@ struct CubinResult { std::vector option_values; CUlinkState link_state; size_t cubin_size; + + std::string info_log; + std::string error_log; + size_t jit_wall_time_idx; + + CubinResult(); + inline float jitWallTime() const { + return *reinterpret_cast(&option_values[jit_wall_time_idx]); + } }; /** diff --git a/Tests/GpuSharedMemoryTest.cpp b/Tests/GpuSharedMemoryTest.cpp index f165ca401f..56cba60933 100644 --- a/Tests/GpuSharedMemoryTest.cpp +++ b/Tests/GpuSharedMemoryTest.cpp @@ -182,23 +182,18 @@ std::unique_ptr compile_and_link_gpu_code( const auto ptx = CodeGenerator::generatePTX(cuda_llir, nvptx_target_machine.get(), context); - auto cubin_result = ptx_to_cubin(ptx, cuda_mgr); - auto& option_keys = cubin_result.option_keys; - auto& option_values = cubin_result.option_values; - auto cubin = cubin_result.cubin; - auto link_state = cubin_result.link_state; - const auto num_options = option_keys.size(); + CubinResult cubin_result = ptx_to_cubin(ptx, cuda_mgr); auto gpu_context = - std::make_unique(cubin, + std::make_unique(cubin_result.cubin, cubin_result.cubin_size, kernel_name, gpu_device_idx, cuda_mgr, - num_options, - &option_keys[0], - &option_values[0]); + cubin_result.option_keys.size(), + cubin_result.option_keys.data(), + cubin_result.option_values.data()); - checkCudaErrors(cuLinkDestroy(link_state)); + checkCudaErrors(cuLinkDestroy(cubin_result.link_state)); return gpu_context; } From 9de6892a67c15d223eceec1732f1be7d75157cb8 Mon Sep 17 00:00:00 2001 From: yoonminnam <53632385+yoonminnam@users.noreply.github.com> Date: Thu, 21 Sep 2023 00:40:21 -0400 Subject: [PATCH 03/72] Fixup window value functions' nullability (#7525) * Fixup window func translator logic * Add test * Re-enable the test query * Refactor window function translation logic * Address comments #1 * Address comment Signed-off-by: Misiu Godfrey --- QueryEngine/RelAlgTranslator.cpp | 265 +++++++++++++++++-------------- QueryEngine/RelAlgTranslator.h | 5 +- QueryEngine/WindowContext.h | 18 +++ QueryEngine/WindowFunctionIR.cpp | 10 ++ Tests/ExecuteTest.cpp | 38 +++++ 5 files changed, 213 insertions(+), 123 deletions(-) diff --git a/QueryEngine/RelAlgTranslator.cpp b/QueryEngine/RelAlgTranslator.cpp index e71944fc0f..a03f76556a 100644 --- a/QueryEngine/RelAlgTranslator.cpp +++ b/QueryEngine/RelAlgTranslator.cpp @@ -2096,8 +2096,93 @@ ExtractField determineTimeUnit(const SQLTypes& window_frame_bound_type, CHECK(false); return kUNKNOWN_FIELD; } + +SqlWindowFrameBoundType determine_frame_bound_type( + const RexWindowFunctionOperator::RexWindowBound& bound) { + if (bound.unbounded) { + CHECK(!bound.bound_expr && !bound.is_current_row); + if (bound.following) { + return SqlWindowFrameBoundType::UNBOUNDED_FOLLOWING; + } else if (bound.preceding) { + return SqlWindowFrameBoundType::UNBOUNDED_PRECEDING; + } + } else { + if (bound.is_current_row) { + CHECK(!bound.unbounded && !bound.bound_expr); + return SqlWindowFrameBoundType::CURRENT_ROW; + } else { + CHECK(!bound.unbounded && bound.bound_expr); + if (bound.following) { + return SqlWindowFrameBoundType::EXPR_FOLLOWING; + } else if (bound.preceding) { + return SqlWindowFrameBoundType::EXPR_PRECEDING; + } + } + } + return SqlWindowFrameBoundType::UNKNOWN; +} + +bool is_negative_framing_bound(const SQLTypes t, + const Datum& d, + bool is_time_unit = false) { + switch (t) { + case kTINYINT: + return d.tinyintval < 0; + case kSMALLINT: + return d.smallintval < 0; + case kINT: + return d.intval < 0; + case kDOUBLE: { + // the only case that double type is used is for handling time interval + // i.e., represent tiny time units like nanosecond and microsecond as the + // equivalent time value with SECOND time unit + CHECK(is_time_unit); + return d.doubleval < 0; + } + case kDECIMAL: + case kNUMERIC: + case kBIGINT: + return d.bigintval < 0; + default: { + throw std::runtime_error( + "We currently only support integer-type literal expression as a window " + "frame bound expression"); + } + } +} + } // namespace +// this function returns three elements as a tuple as follows: +// 1) `bound_expr` is invalid +// 2) `bound_expr` has a negative constant +// 3) a translated bound expr which has `Analyzer::Expr*` type +std::tuple> +RelAlgTranslator::translateFrameBoundExpr(const RexScalar* bound_expr) const { + bool negative_constant = false; + if (dynamic_cast(bound_expr)) { + auto translated_expr = translateScalarRex(bound_expr); + const auto bin_oper = dynamic_cast(translated_expr.get()); + auto time_literal_expr = + dynamic_cast(bin_oper->get_left_operand()); + CHECK(time_literal_expr); + negative_constant = + is_negative_framing_bound(time_literal_expr->get_type_info().get_type(), + time_literal_expr->get_constval(), + true); + return std::make_tuple(false, negative_constant, translated_expr); + } else if (dynamic_cast(bound_expr)) { + auto translated_expr = translateScalarRex(bound_expr); + if (auto literal_expr = + dynamic_cast(translated_expr.get())) { + negative_constant = is_negative_framing_bound( + literal_expr->get_type_info().get_type(), literal_expr->get_constval()); + return std::make_tuple(false, negative_constant, translated_expr); + } + } + return std::make_tuple(true, negative_constant, nullptr); +} + std::shared_ptr RelAlgTranslator::translateWindowFunction( const RexWindowFunctionOperator* rex_window_function) const { std::vector> args; @@ -2119,59 +2204,16 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( auto window_func_kind = rex_window_function->getKind(); if (window_function_is_value(window_func_kind)) { CHECK_GE(args.size(), 1u); - ti = args.front()->get_type_info(); - } - auto determine_frame_bound_type = - [](const RexWindowFunctionOperator::RexWindowBound& bound) { - if (bound.unbounded) { - CHECK(!bound.bound_expr && !bound.is_current_row); - if (bound.following) { - return SqlWindowFrameBoundType::UNBOUNDED_FOLLOWING; - } else if (bound.preceding) { - return SqlWindowFrameBoundType::UNBOUNDED_PRECEDING; - } - } else { - if (bound.is_current_row) { - CHECK(!bound.unbounded && !bound.bound_expr); - return SqlWindowFrameBoundType::CURRENT_ROW; - } else { - CHECK(!bound.unbounded && bound.bound_expr); - if (bound.following) { - return SqlWindowFrameBoundType::EXPR_FOLLOWING; - } else if (bound.preceding) { - return SqlWindowFrameBoundType::EXPR_PRECEDING; - } - } - } - return SqlWindowFrameBoundType::UNKNOWN; - }; - auto is_negative_framing_bound = - [](const SQLTypes t, const Datum& d, bool is_time_unit = false) { - switch (t) { - case kTINYINT: - return d.tinyintval < 0; - case kSMALLINT: - return d.smallintval < 0; - case kINT: - return d.intval < 0; - case kDOUBLE: { - // the only case that double type is used is for handling time interval - // i.e., represent tiny time units like nanosecond and microsecond as the - // equivalent time value with SECOND time unit - CHECK(is_time_unit); - return d.doubleval < 0; - } - case kDECIMAL: - case kNUMERIC: - case kBIGINT: - return d.bigintval < 0; - default: { - throw std::runtime_error( - "We currently only support integer-type literal expression as a window " - "frame bound expression"); - } - } - }; + if (!window_function_is_value_with_frame(window_func_kind)) { + // value window functions w/ frame have logic to access argument's typeinfo + // during codegen, i.e., codegenWindowNavigationFunctionOnFrame(...) + // but not for non-framed value window function, so we use their arg's typeinfo + // as window function's typeinfo + ti = args.front()->get_type_info(); + } + // set value type window functions' nullability + ti.set_notnull(false); + } bool negative_constant = false; bool detect_invalid_frame_start_bound_expr = false; @@ -2200,42 +2242,17 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( has_framing_clause = false; } } else { - auto translate_frame_bound_expr = [&](const RexScalar* bound_expr) { - std::shared_ptr translated_expr; - const auto rex_oper = dynamic_cast(bound_expr); - if (rex_oper && rex_oper->getType().is_timeinterval()) { - translated_expr = translateScalarRex(rex_oper); - const auto bin_oper = - dynamic_cast(translated_expr.get()); - auto time_literal_expr = - dynamic_cast(bin_oper->get_left_operand()); - CHECK(time_literal_expr); - negative_constant = - is_negative_framing_bound(time_literal_expr->get_type_info().get_type(), - time_literal_expr->get_constval(), - true); - return std::make_pair(false, translated_expr); - } - if (dynamic_cast(bound_expr)) { - translated_expr = translateScalarRex(bound_expr); - if (auto literal_expr = - dynamic_cast(translated_expr.get())) { - negative_constant = is_negative_framing_bound( - literal_expr->get_type_info().get_type(), literal_expr->get_constval()); - return std::make_pair(false, translated_expr); - } - } - return std::make_pair(true, translated_expr); - }; - if (frame_start_bound.bound_expr) { - std::tie(detect_invalid_frame_start_bound_expr, frame_start_bound_expr) = - translate_frame_bound_expr(frame_start_bound.bound_expr.get()); + std::tie(detect_invalid_frame_start_bound_expr, + negative_constant, + frame_start_bound_expr) = + translateFrameBoundExpr(frame_start_bound.bound_expr.get()); } if (frame_end_bound.bound_expr) { - std::tie(detect_invalid_frame_end_bound_expr, frame_end_bound_expr) = - translate_frame_bound_expr(frame_end_bound.bound_expr.get()); + std::tie( + detect_invalid_frame_end_bound_expr, negative_constant, frame_end_bound_expr) = + translateFrameBoundExpr(frame_end_bound.bound_expr.get()); } // currently we only support literal expression as frame bound expression @@ -2252,27 +2269,20 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( "A constant expression for window framing should have nonnegative value."); } - auto handle_time_interval_expr_if_necessary = [&](const Analyzer::Expr* bound_expr, - SqlWindowFrameBoundType bound_type, - bool for_start_bound) { - if (bound_expr && bound_expr->get_type_info().is_timeinterval()) { - const auto bound_bin_oper = dynamic_cast(bound_expr); - CHECK(bound_bin_oper->get_optype() == kMULTIPLY); - auto translated_expr = translateIntervalExprForWindowFraming( - order_keys.front(), - bound_type == SqlWindowFrameBoundType::EXPR_PRECEDING, - bound_bin_oper); - if (for_start_bound) { - frame_start_bound_expr = translated_expr; - } else { - frame_end_bound_expr = translated_expr; - } - } - }; - handle_time_interval_expr_if_necessary( - frame_start_bound_expr.get(), frame_start_bound_type, true); - handle_time_interval_expr_if_necessary( - frame_end_bound_expr.get(), frame_end_bound_type, false); + if (frame_start_bound_expr && + frame_start_bound_expr->get_type_info().is_timeinterval()) { + frame_start_bound_expr = translateIntervalExprForWindowFraming( + order_keys.front(), + frame_start_bound_type == SqlWindowFrameBoundType::EXPR_PRECEDING, + frame_start_bound_expr.get()); + } + + if (frame_end_bound_expr && frame_end_bound_expr->get_type_info().is_timeinterval()) { + frame_end_bound_expr = translateIntervalExprForWindowFraming( + order_keys.front(), + frame_end_bound_type == SqlWindowFrameBoundType::EXPR_PRECEDING, + frame_end_bound_expr.get()); + } } if (frame_start_bound.following) { @@ -2284,25 +2294,28 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( "Window framing starting from following row cannot have preceding rows."); } } + if (frame_start_bound.is_current_row && frame_end_bound.preceding && !frame_end_bound.unbounded && has_end_bound_frame_expr) { throw std::runtime_error( "Window framing starting from current row cannot have preceding rows."); } + + if (!frame_start_bound_expr && + frame_start_bound_type == SqlWindowFrameBoundType::UNBOUNDED_PRECEDING && + !frame_end_bound_expr && + frame_end_bound_type == SqlWindowFrameBoundType::CURRENT_ROW) { + has_framing_clause = false; + VLOG(1) << "Ignore range framing mode with a frame bound between " + "UNBOUNDED_PRECEDING and CURRENT_ROW"; + } + if (has_framing_clause) { if (frame_mode == Analyzer::WindowFunction::FrameBoundType::RANGE) { if (order_keys.size() != 1) { throw std::runtime_error( "Window framing with range mode requires a single order-by column"); } - if (!frame_start_bound_expr && - frame_start_bound_type == SqlWindowFrameBoundType::UNBOUNDED_PRECEDING && - !frame_end_bound_expr && - frame_end_bound_type == SqlWindowFrameBoundType::CURRENT_ROW) { - has_framing_clause = false; - VLOG(1) << "Ignore range framing mode with a frame bound between " - "UNBOUNDED_PRECEDING and CURRENT_ROW"; - } std::set colvar_set(Analyzer::ColumnVar::colvar_comp); @@ -2317,11 +2330,19 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( } } } + auto const func_name = ::toString(window_func_kind); auto const num_args = args.size(); bool need_order_by_clause = false; bool need_frame_def = false; switch (window_func_kind) { + case SqlWindowFunctionKind::COUNT: { + if (has_framing_clause && args.empty()) { + args.push_back( + makeExpr(g_bigint_count ? kBIGINT : kINT, true)); + } + break; + } case SqlWindowFunctionKind::LEAD_IN_FRAME: case SqlWindowFunctionKind::LAG_IN_FRAME: { need_order_by_clause = true; @@ -2402,9 +2423,6 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( if (num_args != 2) { throw std::runtime_error(func_name + " has an invalid number of input arguments"); } - // NTH_VALUE(_IN_FRAME) may return null value even if the argument is non-null - // column - ti.set_notnull(false); if (window_func_kind == SqlWindowFunctionKind::NTH_VALUE_IN_FRAME) { need_order_by_clause = true; need_frame_def = true; @@ -2452,22 +2470,22 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( break; default:; } + if (need_order_by_clause && order_keys.empty()) { throw std::runtime_error(func_name + " requires an ORDER BY clause"); } + if (need_frame_def && !has_framing_clause) { throw std::runtime_error(func_name + " requires window frame definition"); } + if (!has_framing_clause) { frame_start_bound_type = SqlWindowFrameBoundType::UNKNOWN; frame_end_bound_type = SqlWindowFrameBoundType::UNKNOWN; frame_start_bound_expr = nullptr; frame_end_bound_expr = nullptr; } - if (window_func_kind == SqlWindowFunctionKind::COUNT && has_framing_clause && - args.empty()) { - args.push_back(makeExpr(g_bigint_count ? kBIGINT : kINT, true)); - } + return makeExpr( ti, rex_window_function->getKind(), @@ -2483,10 +2501,13 @@ std::shared_ptr RelAlgTranslator::translateWindowFunction( std::shared_ptr RelAlgTranslator::translateIntervalExprForWindowFraming( std::shared_ptr order_key, bool for_preceding_bound, - const Analyzer::BinOper* frame_bound_expr) const { + const Analyzer::Expr* expr) const { // translate time interval expression and prepare appropriate frame bound expression: // a) manually compute time unit datum: time type // b) use dateadd expression: date and timestamp + const auto frame_bound_expr = dynamic_cast(expr); + CHECK(frame_bound_expr); + CHECK_EQ(frame_bound_expr->get_optype(), kMULTIPLY); const auto order_key_ti = order_key->get_type_info(); const auto frame_bound_ti = frame_bound_expr->get_type_info(); const auto time_val_expr = diff --git a/QueryEngine/RelAlgTranslator.h b/QueryEngine/RelAlgTranslator.h index 86f222c04b..706c41a99b 100644 --- a/QueryEngine/RelAlgTranslator.h +++ b/QueryEngine/RelAlgTranslator.h @@ -161,10 +161,13 @@ class RelAlgTranslator { std::shared_ptr translateWindowFunction( const RexWindowFunctionOperator*) const; + std::tuple> translateFrameBoundExpr( + const RexScalar* bound_expr) const; + std::shared_ptr translateIntervalExprForWindowFraming( std::shared_ptr order_key, bool for_preceding_bound, - const Analyzer::BinOper* frame_bound_expr) const; + const Analyzer::Expr* expr) const; Analyzer::ExpressionPtrVector translateFunctionArgs(const RexFunctionOperator*) const; diff --git a/QueryEngine/WindowContext.h b/QueryEngine/WindowContext.h index f9dde393ca..dd971648c9 100644 --- a/QueryEngine/WindowContext.h +++ b/QueryEngine/WindowContext.h @@ -33,6 +33,24 @@ inline bool window_function_is_value(const SqlWindowFunctionKind kind) { case SqlWindowFunctionKind::FIRST_VALUE: case SqlWindowFunctionKind::LAST_VALUE: case SqlWindowFunctionKind::NTH_VALUE: + case SqlWindowFunctionKind::LAG_IN_FRAME: + case SqlWindowFunctionKind::LEAD_IN_FRAME: + case SqlWindowFunctionKind::FIRST_VALUE_IN_FRAME: + case SqlWindowFunctionKind::LAST_VALUE_IN_FRAME: + case SqlWindowFunctionKind::NTH_VALUE_IN_FRAME: + return true; + default: + return false; + } +} + +inline bool window_function_is_value_with_frame(const SqlWindowFunctionKind kind) { + switch (kind) { + case SqlWindowFunctionKind::LAG_IN_FRAME: + case SqlWindowFunctionKind::LEAD_IN_FRAME: + case SqlWindowFunctionKind::FIRST_VALUE_IN_FRAME: + case SqlWindowFunctionKind::LAST_VALUE_IN_FRAME: + case SqlWindowFunctionKind::NTH_VALUE_IN_FRAME: return true; default: return false; diff --git a/QueryEngine/WindowFunctionIR.cpp b/QueryEngine/WindowFunctionIR.cpp index 2f3a58d51b..dec6c123e8 100644 --- a/QueryEngine/WindowFunctionIR.cpp +++ b/QueryEngine/WindowFunctionIR.cpp @@ -182,8 +182,14 @@ llvm::Value* get_null_value_by_size(CgenState* cgen_state, SQLTypeInfo col_ti) { case kTINYINT: return cgen_state->llInt((int8_t)inline_int_null_value()); case kSMALLINT: + if (col_ti.get_compression() == kENCODING_FIXED) { + return cgen_state->llInt((int16_t)(inline_fixed_encoding_null_val(col_ti))); + } return cgen_state->llInt((int16_t)inline_int_null_value()); case kINT: + if (col_ti.get_compression() == kENCODING_FIXED) { + return cgen_state->llInt((int32_t)(inline_fixed_encoding_null_val(col_ti))); + } return cgen_state->llInt((int32_t)inline_int_null_value()); case kTIME: case kTIMESTAMP: @@ -191,6 +197,10 @@ llvm::Value* get_null_value_by_size(CgenState* cgen_state, SQLTypeInfo col_ti) { return cgen_state->llInt((int64_t)(inline_fixed_encoding_null_val(col_ti))); } case kBIGINT: + if (col_ti.get_compression() == kENCODING_FIXED) { + return cgen_state->llInt((int64_t)(inline_fixed_encoding_null_val(col_ti))); + } + return cgen_state->llInt((int64_t)inline_int_null_value()); case kINTERVAL_DAY_TIME: case kINTERVAL_YEAR_MONTH: case kDECIMAL: diff --git a/Tests/ExecuteTest.cpp b/Tests/ExecuteTest.cpp index 5fabb1f51a..dabbc7b6f7 100644 --- a/Tests/ExecuteTest.cpp +++ b/Tests/ExecuteTest.cpp @@ -22255,6 +22255,17 @@ TEST_F(Select, WindowFunctionEmptyPartitions) { " GROUP BY DATE_TRUNC(DAY, d), x ORDER BY DATE_TRUNC(DAY, d) NULLS FIRST;"; EXPECT_NO_THROW(run_multiple_agg(query, dt)); } + + { + std::string query = + "SELECT DATE_TRUNC(DAY, d) AS binned_day, COUNT(*) AS n, SUM(x) AS sum_x, " + "COUNT(*) - LAG(COUNT(*)) OVER ( ORDER BY DATE_TRUNC(DAY, d) ) AS " + "lag_n_order_by_d, SUM(x) / SUM(SUM(x+1)) OVER ( ORDER BY DATE_TRUNC(DAY, d)) " + "AS sum_over_lag_sum_x FROM " + + table_name + + " GROUP BY DATE_TRUNC(DAY, d) ORDER BY DATE_TRUNC(DAY, d) NULLS FIRST;"; + EXPECT_NO_THROW(run_multiple_agg(query, dt)); + } } } @@ -22481,6 +22492,16 @@ TEST_F(Select, WindowFunctionLag) { } } } + + { + std::string q1 = "SELECT LAG(COUNT(*), 1) OVER (ORDER BY x ASC NULLS FIRST) FROM " + + table_name + " GROUP BY x ORDER BY x ASC NULLS FIRST;"; + c(q1, q1, dt); + std::string q2 = + "SELECT COUNT(*) - LAG(COUNT(*), 1) OVER (ORDER BY x ASC NULLS FIRST) FROM " + + table_name + " GROUP BY x ORDER BY x ASC NULLS FIRST;"; + c(q2, q2, dt); + } } } @@ -22585,6 +22606,16 @@ TEST_F(Select, WindowFunctionLead) { } } } + { + std::string q1 = + "SELECT LEAD(COUNT(*), 1) OVER (ORDER BY x ASC NULLS FIRST) FROM " + + table_name + " GROUP BY x ORDER BY x ASC NULLS FIRST;"; + c(q1, q1, dt); + std::string q2 = + "SELECT COUNT(*) - LEAD(COUNT(*), 1) OVER (ORDER BY x ASC NULLS FIRST) FROM " + + table_name + " GROUP BY x ORDER BY x ASC NULLS FIRST;"; + c(q2, q2, dt); + } } } @@ -22969,6 +23000,13 @@ TEST_F(Select, WindowFunctionSum) { table_name + ")) ORDER BY total ASC NULLS FIRST"; c(query, query, dt); } + { + std::string query = + "SELECT x, COUNT(*) - SUM(COUNT(*)) OVER (ORDER BY COUNT(*) DESC, x NULLS " + "FIRST) FROM " + + table_name + " GROUP BY x ORDER BY COUNT(*) DESC, x NULLS FIRST;"; + c(query, query, dt); + } } } From 2d3be011d2df0c454f1e368a23ae76c19cf4fe3f Mon Sep 17 00:00:00 2001 From: jack-mapd Date: Thu, 21 Sep 2023 10:39:58 +0200 Subject: [PATCH 04/72] Add extra version info to dependencies (#7431) * Add extra version info to dependencies and copy dependency version file to build Signed-off-by: Misiu Godfrey --- CMakeLists.txt | 25 +++++++++++++++++++++++ scripts/common-functions.sh | 24 ++++++++++++++-------- scripts/heavyai-dependency-tar-builder.sh | 18 ++++++++++++++-- 3 files changed, 57 insertions(+), 10 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 30e5cb5211..0cd374d483 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -134,6 +134,30 @@ macro(set_alternate_linker linker) endif() endmacro() +macro(InstallVersionFile) + find_file(MapdDepsVersion_FILE mapd_deps_version.txt NO_CACHE) + + if(NOT MapdDepsVersion_FILE) + message(WARNING "Build could NOT find deps version file mapd_deps_version.txt") + else() + message(STATUS "Found deps version file ${MapdDepsVersion_FILE}") + file(MAKE_DIRECTORY ${CMAKE_BINARY_DIR}/version) + set(HEAVY_DEPS_CUSTOM ${CMAKE_BINARY_DIR}/version/heavyai_deps_version.txt) + # cp mapd_version data to heavyai deps, removing un-tagged lines. + add_custom_command(OUTPUT ${HEAVY_DEPS_CUSTOM} + DEPENDS ${MapdDepsVersion_FILE} + COMMAND ${CMAKE_COMMAND} -E copy ${MapdDepsVersion_FILE} ${HEAVY_DEPS_CUSTOM} + # In the copied deps files leave the first line with the deps generated info + # and any other line starting with 'Public Release:', though remove the public release tag + VERBATIM + COMMAND "sed" "-i" "-n" "/^Public Release:/s/^Public Release://p" ${HEAVY_DEPS_CUSTOM}) + + add_custom_target(HeavyDepsVersionTarget DEPENDS ${HEAVY_DEPS_CUSTOM}) + add_dependencies(heavydb HeavyDepsVersionTarget) + install(FILES ${HEAVY_DEPS_CUSTOM} DESTINATION "." COMPONENT "doc") + endif() +endmacro() + set(USE_ALTERNATE_LINKER "" CACHE STRING "Use alternate linker. Leave empty for system default; alternatives are 'gold', 'lld', 'bfd', 'mold'") if(NOT "${USE_ALTERNATE_LINKER}" STREQUAL "") set_alternate_linker(${USE_ALTERNATE_LINKER}) @@ -1107,6 +1131,7 @@ endif() add_executable(heavydb HeavyDB.cpp ${CMAKE_BINARY_DIR}/MapDRelease.h) set_target_properties(heavydb PROPERTIES COMPILE_DEFINITIONS "${TIME_LIMITED_DEFINITIONS}") +InstallVersionFile() add_custom_command( DEPENDS ${CMAKE_SOURCE_DIR}/heavy.thrift diff --git a/scripts/common-functions.sh b/scripts/common-functions.sh index cdebc2418c..70f35f6f2e 100755 --- a/scripts/common-functions.sh +++ b/scripts/common-functions.sh @@ -15,15 +15,23 @@ if [ "$NOCUDA" = "true" ]; then fi function generate_deps_version_file() { - # get the git hash - pushd $SCRIPTS_DIR >/dev/null - # For some version of git when run by Jenkins - # this command will fail and return an error. - # Better to carry on report what is known. - SHORT_HASH=$(git rev-parse --short HEAD || echo "UNKOWN") - popd >/dev/null - echo "Deps generated for git hash [$SHORT_HASH] and SUFFIX [$SUFFIX]" > $PREFIX/mapd_deps_version.txt + # SUFFIX, BRANCH_NAME, GIT_COMMIT and BUILD_CONTAINER_NAME are set as environment variables not as parameters and + # are generally set 'on' the calling docker container. + echo "Public Release:Deps generated for prefix [$PREFIX], commit [$GIT_COMMIT] and SUFFIX [$SUFFIX]" > $PREFIX/mapd_deps_version.txt + # BUILD_CONTAINER_IMAGE will only be set if called from heavyai-dependency-tar-builder.sh + if [[ -n $BUILD_CONTAINER_IMAGE_ID ]] ; then + echo "Public Release:Using build image id [${BUILD_CONTAINER_IMAGE_ID}]" >> $PREFIX/mapd_deps_version.txt + fi + if [[ -n $BUILD_CONTAINER_IMAGE ]] ; then + # Not copied to released version of this file + echo "Using build image [${BUILD_CONTAINER_IMAGE}]" >> $PREFIX/mapd_deps_version.txt + fi + echo "Component version information:" >> $PREFIX/mapd_deps_version.txt # Grab all the _VERSION variables and print them to the file + # This isn't a complete list of all software and versions. For example openssl either uses + # the version that ships with the OS or it is installed from the OS specific file and + # doesn't use an _VERSION variable. + # Not to be copied to released version of this file for i in $(compgen -A variable | grep _VERSION) ; do echo $i "${!i}" ; done >> $PREFIX/mapd_deps_version.txt } diff --git a/scripts/heavyai-dependency-tar-builder.sh b/scripts/heavyai-dependency-tar-builder.sh index 62a0414fa2..32c31e06ae 100755 --- a/scripts/heavyai-dependency-tar-builder.sh +++ b/scripts/heavyai-dependency-tar-builder.sh @@ -105,15 +105,25 @@ sudo docker pull $BUILD_CONTAINER_IMAGE # strip verion number from os OPERATING_SYSTEM=$(echo $OPERATING_SYSTEM | sed 's/[0-9,\.]*$//') +# +# Note we use two methods to pass run information to the docker container. +# Firstly via options on the command the docker container runs - 'docker_cmd' +# and secondly via environment varibles on the docker command itself (-e options) +# The value set in the environment, with the -e options are intended for use by the +# common-functions.sh script sourced by the 'main' mapd-deps-${OPERATING_SYSTEM} +# script. +# if [[ $OPERATING_SYSTEM == "centos" ]] ; then docker_cmd="yum install sudo -y && ./mapd-deps-${OPERATING_SYSTEM}.sh --savespace --compress $TSAN_PARAM --cache=/dep_cache" else - docker_cmd='echo -e "#!/bin/sh\n\${@}" > /usr/sbin/sudo && chmod +x /usr/sbin/sudo && ./mapd-deps-'${OPERATING_SYSTEM}'.sh --savespace --compress' + docker_cmd='echo -e "#!/bin/sh\n\${@}" > /usr/sbin/sudo && chmod +x /usr/sbin/sudo && ./mapd-deps-'${OPERATING_SYSTEM}'.sh --savespace --compress --cache=/dep_cache' fi PACKAGE_CACHE=/theHoard/export/dep_cache echo "Running [$docker_cmd] in $BUILD_CONTAINER_IMAGE" - +BUILD_CONTAINER_IMAGE_ID=$(docker images -q $BUILD_CONTAINER_IMAGE --no-trunc) +# Note - to log the container image name pass it +# in as an environmemt. sudo docker run --rm --runtime=nvidia \ -v $BUILD_TMP_DIR:/build \ -v $PACKAGE_CACHE:/dep_cache \ @@ -121,6 +131,10 @@ sudo docker run --rm --runtime=nvidia \ -e USER=root \ --memory=64G --cpuset-cpus=$CPU_SET \ -e SUFFIX=${SUFFIX} \ + -e BUILD_CONTAINER_IMAGE_ID=${BUILD_CONTAINER_IMAGE_ID} \ + -e BUILD_CONTAINER_IMAGE=${BUILD_CONTAINER_IMAGE} \ + -e BRANCH_NAME=${BRANCH_NAME} \ + -e GIT_COMMIT=${GIT_COMMIT} \ --name $BUILD_CONTAINER_NAME \ $BUILD_CONTAINER_IMAGE \ bash -c "$docker_cmd" From 4f63cee6ba997aa142f04dc9f01a71faf7075c35 Mon Sep 17 00:00:00 2001 From: Matt Pulver Date: Sat, 23 Sep 2023 16:14:01 -0400 Subject: [PATCH 05/72] Fix queries that contain more than one Type C aggregate. (#7535) Signed-off-by: Misiu Godfrey --- QueryEngine/QueryMemoryInitializer.cpp | 17 ++++++------ Tests/ExecuteTest.cpp | 36 ++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 9 deletions(-) diff --git a/QueryEngine/QueryMemoryInitializer.cpp b/QueryEngine/QueryMemoryInitializer.cpp index 4799cc41c8..77e6063042 100644 --- a/QueryEngine/QueryMemoryInitializer.cpp +++ b/QueryEngine/QueryMemoryInitializer.cpp @@ -773,18 +773,17 @@ void QueryMemoryInitializer::initColumnsPerRow( col_ptr += query_mem_desc.getNextColOffInBytesRowOnly(col_ptr, col_idx++)) { int64_t init_val{0}; if (query_mem_desc.isGroupBy()) { - if (agg_op_metadata.has_count_distinct) { + if (agg_op_metadata.has_count_distinct && + agg_op_metadata.count_distinct_buf_size[col_idx]) { // COUNT DISTINCT / APPROX_COUNT_DISTINCT // create a data structure for count_distinct operator per entries const int64_t bm_sz{agg_op_metadata.count_distinct_buf_size[col_idx]}; - if (bm_sz) { - CHECK_EQ(static_cast(query_mem_desc.getPaddedSlotWidthBytes(col_idx)), - sizeof(int64_t)); - init_val = - bm_sz > 0 ? allocateCountDistinctBitmap(bm_sz) : allocateCountDistinctSet(); - CHECK_NE(init_val, 0); - ++init_vec_idx; - } + CHECK_EQ(static_cast(query_mem_desc.getPaddedSlotWidthBytes(col_idx)), + sizeof(int64_t)); + init_val = + bm_sz > 0 ? allocateCountDistinctBitmap(bm_sz) : allocateCountDistinctSet(); + CHECK_NE(init_val, 0); + ++init_vec_idx; } else if (agg_op_metadata.has_tdigest && agg_op_metadata.quantile_params[col_idx]) { auto const q = *agg_op_metadata.quantile_params[col_idx]; diff --git a/Tests/ExecuteTest.cpp b/Tests/ExecuteTest.cpp index dabbc7b6f7..fea2fac440 100644 --- a/Tests/ExecuteTest.cpp +++ b/Tests/ExecuteTest.cpp @@ -4431,6 +4431,42 @@ TEST_F(Select, ModeOrderBy) { } } +TEST_F(Select, TypeCAggregates) { + SKIP_ALL_ON_AGGREGATOR(); // APPROX_MEDIAN() is not supported in distributed mode. + for (auto dt : {ExecutorDeviceType::CPU, ExecutorDeviceType::GPU}) { + SKIP_NO_GPU(); + // GROUP BY + ORDER BY aggregate + c("SELECT w, APPROX_MEDIAN(x), MODE(y) FROM test GROUP BY w" + " ORDER BY COUNT(DISTINCT z);", + "SELECT * FROM (VALUES (-8, 7.0, 42), (-7, 7.5, 43));", + dt); + // Non-group-by aggregate + c("SELECT APPROX_MEDIAN(x), COUNT(DISTINCT y), MODE(z) FROM test;", + "SELECT * FROM (VALUES (7.0, 2, 101));", + dt); + // Non-group-by aggregate w/ JOIN ON + c("SELECT APPROX_MEDIAN(test.x), MODE(test.y), COUNT(DISTINCT test.z)" + " FROM test JOIN test_inner ON test.y=test_inner.y;", + "SELECT * FROM (VALUES (7.5, 43, 2));", + dt); + // Non-group-by aggregate w/ CROSS JOIN filter + c("SELECT APPROX_MEDIAN(test.x), MODE(test.y), COUNT(DISTINCT test.z)" + " FROM test, test_inner WHERE test.y=test_inner.y;", + "SELECT * FROM (VALUES (7.5, 43, 2));", + dt); + // GROUP BY w/ JOIN ON + c("SELECT APPROX_MEDIAN(test.w), test.x, MODE(test.y), COUNT(DISTINCT test.z) FROM" + " test JOIN test_inner ON test.y=test_inner.y GROUP BY test.x ORDER BY test.x;", + "SELECT * FROM (VALUES (-7.0, 7, 43, 1), (-7.0, 8, 43, 1));", + dt); + // GROUP BY w/ CROSS JOIN filter + c("SELECT APPROX_MEDIAN(test.w), test.x, MODE(test.y), COUNT(DISTINCT test.z) FROM" + " test, test_inner WHERE test.y=test_inner.y GROUP BY test.x ORDER BY test.x;", + "SELECT * FROM (VALUES (-7.0, 7, 43, 1), (-7.0, 8, 43, 1));", + dt); + } +} + TEST_F(Select, ScanNoAggregation) { for (auto dt : {ExecutorDeviceType::CPU, ExecutorDeviceType::GPU}) { SKIP_NO_GPU(); From 20905ef050b8b66590da389dc1c2fb34a66ac3f4 Mon Sep 17 00:00:00 2001 From: Matt Pulver Date: Mon, 25 Sep 2023 13:47:11 -0400 Subject: [PATCH 06/72] Add URL_ENCODE and URL_DECODE string functions (#7531) Signed-off-by: Misiu Godfrey --- Analyzer/Analyzer.cpp | 10 ++ Analyzer/Analyzer.h | 60 ++++++++++++ QueryEngine/RelAlgTranslator.cpp | 6 ++ Shared/sqldefs.h | 12 +++ StringOps/StringOps.cpp | 80 ++++++++++++++++ StringOps/StringOps.h | 14 +++ Tests/StringFunctionsTest.cpp | 95 +++++++++++++++++++ .../parser/HeavyDBSqlOperatorTable.java | 48 ++++++++++ 8 files changed, 325 insertions(+) diff --git a/Analyzer/Analyzer.cpp b/Analyzer/Analyzer.cpp index a6d3bed182..5a68167db7 100644 --- a/Analyzer/Analyzer.cpp +++ b/Analyzer/Analyzer.cpp @@ -3778,6 +3778,16 @@ std::shared_ptr Base64DecodeStringOper::deep_copy() const { std::dynamic_pointer_cast(StringOper::deep_copy())); } +std::shared_ptr UrlEncodeStringOper::deep_copy() const { + return makeExpr( + std::dynamic_pointer_cast(StringOper::deep_copy())); +} + +std::shared_ptr UrlDecodeStringOper::deep_copy() const { + return makeExpr( + std::dynamic_pointer_cast(StringOper::deep_copy())); +} + std::shared_ptr TryStringCastOper::deep_copy() const { return makeExpr( std::dynamic_pointer_cast(StringOper::deep_copy())); diff --git a/Analyzer/Analyzer.h b/Analyzer/Analyzer.h index 27237fe7f1..3944384d93 100644 --- a/Analyzer/Analyzer.h +++ b/Analyzer/Analyzer.h @@ -2449,6 +2449,66 @@ class Base64DecodeStringOper : public StringOper { std::vector getArgNames() const override { return {"operand"}; } }; +class UrlEncodeStringOper : public StringOper { + public: + UrlEncodeStringOper(const std::shared_ptr& operand) + : StringOper(SqlStringOpKind::URL_ENCODE, + {operand}, + getMinArgs(), + getExpectedTypeFamilies(), + getArgNames()) {} + + UrlEncodeStringOper(const std::vector>& operands) + : StringOper(SqlStringOpKind::URL_ENCODE, + operands, + getMinArgs(), + getExpectedTypeFamilies(), + getArgNames()) {} + + UrlEncodeStringOper(const std::shared_ptr& string_oper) + : StringOper(string_oper) {} + + std::shared_ptr deep_copy() const override; + + size_t getMinArgs() const override { return 1u; } + + std::vector getExpectedTypeFamilies() const override { + return {OperandTypeFamily::STRING_FAMILY}; + } + + std::vector getArgNames() const override { return {"operand"}; } +}; + +class UrlDecodeStringOper : public StringOper { + public: + UrlDecodeStringOper(const std::shared_ptr& operand) + : StringOper(SqlStringOpKind::URL_DECODE, + {operand}, + getMinArgs(), + getExpectedTypeFamilies(), + getArgNames()) {} + + UrlDecodeStringOper(const std::vector>& operands) + : StringOper(SqlStringOpKind::URL_DECODE, + operands, + getMinArgs(), + getExpectedTypeFamilies(), + getArgNames()) {} + + UrlDecodeStringOper(const std::shared_ptr& string_oper) + : StringOper(string_oper) {} + + std::shared_ptr deep_copy() const override; + + size_t getMinArgs() const override { return 1u; } + + std::vector getExpectedTypeFamilies() const override { + return {OperandTypeFamily::STRING_FAMILY}; + } + + std::vector getArgNames() const override { return {"operand"}; } +}; + class TryStringCastOper : public StringOper { public: TryStringCastOper(const SQLTypeInfo& ti, const std::shared_ptr& operand) diff --git a/QueryEngine/RelAlgTranslator.cpp b/QueryEngine/RelAlgTranslator.cpp index a03f76556a..2354d28413 100644 --- a/QueryEngine/RelAlgTranslator.cpp +++ b/QueryEngine/RelAlgTranslator.cpp @@ -1529,6 +1529,10 @@ std::shared_ptr RelAlgTranslator::translateStringOper( return makeExpr(args); case SqlStringOpKind::LEVENSHTEIN_DISTANCE: return makeExpr(args); + case SqlStringOpKind::URL_ENCODE: + return makeExpr(args); + case SqlStringOpKind::URL_DECODE: + return makeExpr(args); default: { throw std::runtime_error("Unsupported string function."); } @@ -1788,6 +1792,8 @@ std::shared_ptr RelAlgTranslator::translateFunction( "JSON_VALUE"sv, "BASE64_ENCODE"sv, "BASE64_DECODE"sv, + "URL_ENCODE"sv, + "URL_DECODE"sv, "TRY_CAST"sv, "POSITION"sv, "JAROWINKLER_SIMILARITY"sv, diff --git a/Shared/sqldefs.h b/Shared/sqldefs.h index f525339e3a..22da6a306b 100644 --- a/Shared/sqldefs.h +++ b/Shared/sqldefs.h @@ -112,6 +112,8 @@ enum class SqlStringOpKind { JSON_VALUE, BASE64_ENCODE, BASE64_DECODE, + URL_ENCODE, + URL_DECODE, TRY_STRING_CAST, // string-to-numeric POSITION, // string-to-numeric JAROWINKLER_SIMILARITY, // string-to-numeric @@ -369,6 +371,10 @@ inline std::ostream& operator<<(std::ostream& os, const SqlStringOpKind kind) { return os << "BASE64_ENCODE"; case SqlStringOpKind::BASE64_DECODE: return os << "BASE64_DECODE"; + case SqlStringOpKind::URL_ENCODE: + return os << "URL_ENCODE"; + case SqlStringOpKind::URL_DECODE: + return os << "URL_DECODE"; case SqlStringOpKind::TRY_STRING_CAST: return os << "TRY_STRING_CAST"; case SqlStringOpKind::POSITION: @@ -449,6 +455,12 @@ inline SqlStringOpKind name_to_string_op_kind(const std::string& func_name) { if (func_name == "BASE64_DECODE") { return SqlStringOpKind::BASE64_DECODE; } + if (func_name == "URL_ENCODE") { + return SqlStringOpKind::URL_ENCODE; + } + if (func_name == "URL_DECODE") { + return SqlStringOpKind::URL_DECODE; + } if (func_name == "TRY_CAST") { return SqlStringOpKind::TRY_STRING_CAST; } diff --git a/StringOps/StringOps.cpp b/StringOps/StringOps.cpp index c7eb852c2f..3751683c52 100644 --- a/StringOps/StringOps.cpp +++ b/StringOps/StringOps.cpp @@ -832,6 +832,78 @@ NullableStrType Base64Decode::operator()(const std::string& str) const { return shared::decode_base64(str); } +namespace { +// Unreserved characters https://www.rfc-editor.org/rfc/rfc3986#section-2.3 +bool is_normal(char const c) { + return std::isalnum(c) || c == '-' || c == '.' || c == '_' || c == '~'; +} + +// True iff c will be encoded into a single character. +bool is_singular(char const c) { + return is_normal(c) || c == ' '; +} + +// Count % chars that are eligible to begin a url-encoded triplet. +size_t count_percents(std::string const& str) { + size_t n_percents = 0u; + if (2u < str.size()) { + for (size_t i = 0u; i < str.size() - 2u; ++i) { + if (str[i] == '%') { + ++n_percents; + i += 2u; + } + } + } + return n_percents; +} + +// If hex is a hex digit, return int value from 0-15. Otherwise undefined. +int nibble(char const hex) { + return 'A' <= hex ? std::toupper(hex) + (10 - 'A') : hex - '0'; +} +} // namespace + +// Encode unreserved characters (RFC 3986 Sec. 2.3) into themselves. +// Encode space ' ' into plus '+'. +// Encode all other characters c into "%XX" where XX is the hex value of c. +NullableStrType UrlEncode::operator()(const std::string& str) const { + constexpr char const* tr = "0123456789ABCDEF"; + // Number of characters in string that will be copied/translated into a single char. + size_t const n_singular = std::count_if(str.begin(), str.end(), is_singular); + std::string encoded; + encoded.reserve(str.size() + 2u * (str.size() - n_singular)); + for (char const c : str) { + if (is_normal(c)) { + encoded.append(1u, c); + } else if (c == ' ') { + encoded.append(1u, '+'); + } else { + encoded.append(1u, '%'); + encoded.append(1u, tr[(c >> 4) & 0xf]); + encoded.append(1u, tr[c & 0xf]); + } + } + return encoded; +} + +// Inverse of UrlEncode::operator(). Garbage in, garbage out, but must never segfault. +NullableStrType UrlDecode::operator()(const std::string& str) const { + size_t const n_percents = count_percents(str); + std::string decoded; + decoded.reserve(str.size() - 2u * n_percents); + for (size_t i = 0u; i < str.size(); ++i) { + if (str[i] == '%' && i + 2u < str.size()) { + decoded.append(1u, nibble(str[i + 1u]) << 4 ^ nibble(str[i + 2u])); + i += 2u; // Skip the two hexadecimal digits + } else if (str[i] == '+') { + decoded.append(1u, ' '); + } else { // Append normal characters, or % if one of last two characters. + decoded.append(1u, str[i]); + } + } + return decoded; +} + std::string StringOps::operator()(const std::string& str) const { NullableStrType modified_str(str); if (modified_str.is_null) { @@ -1081,6 +1153,14 @@ std::unique_ptr gen_string_op(const StringOpInfo& string_op_info CHECK_EQ(num_non_variable_literals, 0UL); return std::make_unique(var_string_optional_literal); } + case SqlStringOpKind::URL_ENCODE: { + CHECK_EQ(num_non_variable_literals, 0UL); + return std::make_unique(var_string_optional_literal); + } + case SqlStringOpKind::URL_DECODE: { + CHECK_EQ(num_non_variable_literals, 0UL); + return std::make_unique(var_string_optional_literal); + } case SqlStringOpKind::TRY_STRING_CAST: { CHECK_EQ(num_non_variable_literals, 0UL); return std::make_unique(return_ti, diff --git a/StringOps/StringOps.h b/StringOps/StringOps.h index 76a68cf517..9ba3b43540 100644 --- a/StringOps/StringOps.h +++ b/StringOps/StringOps.h @@ -577,6 +577,20 @@ struct Base64Decode : public StringOp { NullableStrType operator()(const std::string& str) const override; }; +struct UrlEncode : public StringOp { + UrlEncode(const std::optional& var_str_optional_literal) + : StringOp(SqlStringOpKind::URL_ENCODE, var_str_optional_literal) {} + + NullableStrType operator()(const std::string& str) const override; +}; + +struct UrlDecode : public StringOp { + UrlDecode(const std::optional& var_str_optional_literal) + : StringOp(SqlStringOpKind::URL_DECODE, var_str_optional_literal) {} + + NullableStrType operator()(const std::string& str) const override; +}; + struct NullOp : public StringOp { NullOp(const std::optional& var_str_optional_literal, const SqlStringOpKind op_kind) diff --git a/Tests/StringFunctionsTest.cpp b/Tests/StringFunctionsTest.cpp index b81f3a2a96..ef41772983 100644 --- a/Tests/StringFunctionsTest.cpp +++ b/Tests/StringFunctionsTest.cpp @@ -1635,6 +1635,101 @@ TEST_F(StringFunctionTest, Base64) { } } +class UrlEncodeDecode : public StringFunctionTest { + public: + struct Test { + std::string_view decoded; + std::string_view encoded; + std::string_view test_name; + }; + using Param = std::tuple; + // NOTE: test names must be non-empty, unique, and may only contain ASCII alphanumeric + // characters. [Use underscores carefully.] + static std::string testName(testing::TestParamInfo const& info) { + std::ostringstream oss; + oss << std::get<0>(info.param) << '_' << std::get<1>(info.param).test_name; + return oss.str(); + } +}; + +class UrlEncode : public UrlEncodeDecode, + public testing::WithParamInterface { + public: + void test_encode(ExecutorDeviceType const dt, Test const test) { + std::ostringstream query; + query << "SELECT '" << test.encoded << "' = URL_ENCODE('" << test.decoded << "');"; + EXPECT_TRUE(v(run_simple_agg(query.str(), dt))) << query.str(); + } +}; + +TEST_P(UrlEncode, Test) { + auto const [dt, test] = GetParam(); + if (!skip_tests(dt)) { + test_encode(dt, test); + } +} + +INSTANTIATE_TEST_SUITE_P( + StringFunctionTest, + UrlEncode, + testing::Combine(testing::Values(ExecutorDeviceType::CPU, ExecutorDeviceType::GPU), + testing::Values(UrlEncode::Test{"Hello World!", + "Hello+World%21", + "hello_world"})), + UrlEncode::testName); + +class UrlDecode : public UrlEncodeDecode, + public testing::WithParamInterface { + public: + void test_decode(ExecutorDeviceType const dt, Test const test) { + std::ostringstream query; + query << "SELECT '" << test.decoded << "' = URL_DECODE('" << test.encoded << "');"; + EXPECT_TRUE(v(run_simple_agg(query.str(), dt))) << query.str(); + } +}; + +TEST_P(UrlDecode, Test) { + auto const [dt, test] = GetParam(); + if (!skip_tests(dt)) { + test_decode(dt, test); + } +} + +// If % is one of the last two characters, it should not be decoded by URL_DECODE(). +INSTANTIATE_TEST_SUITE_P( + StringFunctionTest, + UrlDecode, + testing::Combine( + testing::Values(ExecutorDeviceType::CPU, ExecutorDeviceType::GPU), + testing::Values(UrlDecode::Test{"100%", "%3100%", "100_percent"}, + UrlDecode::Test{"100%!", "%3100%!", "100_percent_exclaim"}, + UrlDecode::Test{"100A", "%3100%41", "100A"})), + UrlDecode::testName); + +TEST_F(StringFunctionTest, UrlEncodeAndDecodeInversesAndNull) { + for (auto dt : {ExecutorDeviceType::CPU, ExecutorDeviceType::GPU}) { + SKIP_NO_GPU(); + // Verify URL_DECODE() is inverse of URL_ENCODE() + char const* query = + "SELECT COUNT(*) = COUNT_IF(personal_motto = " + "URL_DECODE(URL_ENCODE(personal_motto))) FROM string_function_test_people;"; + EXPECT_TRUE(v(run_simple_agg(query, dt))) << query; + // Verify empty string and NULL behavior + query = + "SELECT URL_ENCODE(b_str) IS NULL FROM numeric_to_string_test ORDER BY b_str " + "NULLS FIRST LIMIT 1;"; + EXPECT_TRUE(v(run_simple_agg(query, dt))) << query; + query = + "SELECT URL_DECODE(b_str) IS NULL FROM numeric_to_string_test ORDER BY b_str " + "NULLS FIRST LIMIT 1;"; + EXPECT_TRUE(v(run_simple_agg(query, dt))) << query; + query = "SELECT URL_ENCODE('') IS NULL;"; + EXPECT_TRUE(v(run_simple_agg(query, dt))) << query; + query = "SELECT URL_DECODE('') IS NULL;"; + EXPECT_TRUE(v(run_simple_agg(query, dt))) << query; + } +} + TEST_F(StringFunctionTest, TryCastIntegerTypes) { for (auto dt : {ExecutorDeviceType::CPU, ExecutorDeviceType::GPU}) { SKIP_NO_GPU(); diff --git a/java/calcite/src/main/java/com/mapd/calcite/parser/HeavyDBSqlOperatorTable.java b/java/calcite/src/main/java/com/mapd/calcite/parser/HeavyDBSqlOperatorTable.java index 052a8fc93a..1c890ae030 100644 --- a/java/calcite/src/main/java/com/mapd/calcite/parser/HeavyDBSqlOperatorTable.java +++ b/java/calcite/src/main/java/com/mapd/calcite/parser/HeavyDBSqlOperatorTable.java @@ -233,6 +233,8 @@ public void addUDF(final Map extSigs) { addOperator(new RegexpMatch()); addOperator(new Base64Encode()); addOperator(new Base64Decode()); + addOperator(new UrlEncode()); + addOperator(new UrlDecode()); addOperator(new JarowinklerSimilarity()); addOperator(new LevenshteinDistance()); addOperator(new Likely()); @@ -1487,6 +1489,52 @@ public RelDataType inferReturnType(SqlOperatorBinding opBinding) { } } + public static class UrlEncode extends SqlFunction { + public UrlEncode() { + super("URL_ENCODE", + SqlKind.OTHER_FUNCTION, + null, + null, + OperandTypes.family(getSignatureFamilies()), + SqlFunctionCategory.STRING); + } + + private static java.util.List getSignatureFamilies() { + java.util.ArrayList families = + new java.util.ArrayList(); + families.add(SqlTypeFamily.STRING); + return families; + } + + @Override + public RelDataType inferReturnType(SqlOperatorBinding opBinding) { + return opBinding.getOperandType(0); + } + } + + public static class UrlDecode extends SqlFunction { + public UrlDecode() { + super("URL_DECODE", + SqlKind.OTHER_FUNCTION, + null, + null, + OperandTypes.family(getSignatureFamilies()), + SqlFunctionCategory.STRING); + } + + private static java.util.List getSignatureFamilies() { + java.util.ArrayList families = + new java.util.ArrayList(); + families.add(SqlTypeFamily.STRING); + return families; + } + + @Override + public RelDataType inferReturnType(SqlOperatorBinding opBinding) { + return opBinding.getOperandType(0); + } + } + public static class JarowinklerSimilarity extends SqlFunction { public JarowinklerSimilarity() { super("JAROWINKLER_SIMILARITY", From 9434c3766411f6d7afa8f05db963895b82e4fdda Mon Sep 17 00:00:00 2001 From: yoonminnam <53632385+yoonminnam@users.noreply.github.com> Date: Mon, 25 Sep 2023 23:14:10 -0400 Subject: [PATCH 07/72] Refactor cpu memory status logging (#7539) * Refactor cpu memory status logging * Allow loggig for batch insert Signed-off-by: Misiu Godfrey --- QueryRunner/QueryRunner.cpp | 1 - ThriftHandler/DBHandler.cpp | 22 +++++++++++++++++++--- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/QueryRunner/QueryRunner.cpp b/QueryRunner/QueryRunner.cpp index 68664f7cb6..8ddf688417 100644 --- a/QueryRunner/QueryRunner.cpp +++ b/QueryRunner/QueryRunner.cpp @@ -652,7 +652,6 @@ std::shared_ptr QueryRunner::runSQL(const std::string& query_str, return nullptr; } const auto execution_result = runSelectQuery(query_str, std::move(co), std::move(eo)); - VLOG(1) << session_info_->getCatalog().getDataMgr().getSystemMemoryUsage(); return execution_result->getRows(); } diff --git a/ThriftHandler/DBHandler.cpp b/ThriftHandler/DBHandler.cpp index 4e37aa0759..b85f68e135 100644 --- a/ThriftHandler/DBHandler.cpp +++ b/ThriftHandler/DBHandler.cpp @@ -6463,10 +6463,19 @@ void DBHandler::sql_execute_impl(ExecutionResult& _return, std::ostringstream oss; oss << query_substr << post_fix; auto const reduced_query_str = oss.str(); - log_system_cpu_memory_status("Start query execution: " + reduced_query_str, cat); - ScopeGuard cpu_system_memory_logging = [&cat, &reduced_query_str]() { - log_system_cpu_memory_status("Finish query execution: " + reduced_query_str, cat); + bool show_cpu_memory_stat_after_finishing_query = false; + ScopeGuard cpu_system_memory_logging = [&show_cpu_memory_stat_after_finishing_query, + &cat, + &reduced_query_str]() { + if (show_cpu_memory_stat_after_finishing_query) { + log_system_cpu_memory_status("Finish query execution: " + reduced_query_str, cat); + } }; + auto log_cpu_memory_status = + [&reduced_query_str, &cat, &show_cpu_memory_stat_after_finishing_query]() { + log_system_cpu_memory_status("Start query execution: " + reduced_query_str, cat); + show_cpu_memory_stat_after_finishing_query = true; + }; // test to see if db/catalog is writable before execution of a writable SQL/DDL command // TODO: move to execute() (?) @@ -6493,6 +6502,7 @@ void DBHandler::sql_execute_impl(ExecutionResult& _return, CHECK(ddl_query.HasMember("payload")); CHECK(ddl_query["payload"].IsObject()); auto stmt = Parser::InsertIntoTableAsSelectStmt(ddl_query["payload"].GetObject()); + log_cpu_memory_status(); _return.addExecutionTime( measure<>::execution([&]() { stmt.execute(*session_ptr, read_only_); })); return; @@ -6514,6 +6524,7 @@ void DBHandler::sql_execute_impl(ExecutionResult& _return, CHECK(ddl_query.HasMember("payload")); CHECK(ddl_query["payload"].IsObject()); auto stmt = Parser::CreateTableAsSelectStmt(ddl_query["payload"].GetObject()); + log_cpu_memory_status(); _return.addExecutionTime( measure<>::execution([&]() { stmt.execute(*session_ptr, read_only_); })); } @@ -6533,6 +6544,9 @@ void DBHandler::sql_execute_impl(ExecutionResult& _return, CHECK(ddl_query.HasMember("payload")); CHECK(ddl_query["payload"].IsObject()); auto stmt = Parser::InsertValuesStmt(cat, ddl_query["payload"].GetObject()); + if (stmt.get_value_lists().size() > 1) { + log_cpu_memory_status(); + } _return.addExecutionTime( measure<>::execution([&]() { stmt.execute(*session_ptr, read_only_); })); return; @@ -6586,6 +6600,7 @@ void DBHandler::sql_execute_impl(ExecutionResult& _return, _return.addExecutionTime(measure<>::execution( [&]() { execute_distributed_copy_statement(import_stmt, *session_ptr); })); } else { + log_cpu_memory_status(); _return.addExecutionTime(measure<>::execution( [&]() { import_stmt->execute(*session_ptr, read_only_); })); } @@ -6764,6 +6779,7 @@ void DBHandler::sql_execute_impl(ExecutionResult& _return, std::this_thread::sleep_for(std::chrono::milliseconds(10)); } } + log_cpu_memory_status(); dispatch_queue_->submit(execute_rel_alg_task, pw.getDMLType() == ParserWrapper::DMLType::Update || pw.getDMLType() == ParserWrapper::DMLType::Delete); From 0b7515fe257e329777c76dca38093da3467fe22e Mon Sep 17 00:00:00 2001 From: Paul Aiyedun Date: Tue, 19 Sep 2023 16:46:26 -0700 Subject: [PATCH 08/72] Add table option validations * Add table page size validation * Add table string options type validation * Add table options validation tests Signed-off-by: Misiu Godfrey --- DataMgr/FileMgr/FileBuffer.cpp | 31 +++++++---- DataMgr/FileMgr/FileBuffer.h | 6 +- Parser/ParserNode.cpp | 11 +++- Parser/ParserNode.h | 17 +++++- Tests/CachingFileMgrTest.cpp | 2 +- Tests/CreateAndDropTableDdlTest.cpp | 86 ++++++++++++++++++++++++++++- 6 files changed, 133 insertions(+), 20 deletions(-) diff --git a/DataMgr/FileMgr/FileBuffer.cpp b/DataMgr/FileMgr/FileBuffer.cpp index 085a6cfbd1..de0a0bb5dd 100644 --- a/DataMgr/FileMgr/FileBuffer.cpp +++ b/DataMgr/FileMgr/FileBuffer.cpp @@ -46,7 +46,7 @@ FileBuffer::FileBuffer(FileMgr* fm, , chunkKey_(chunkKey) { // Create a new FileBuffer CHECK(fm_); - calcHeaderBuffer(); + setBufferHeaderSize(); CHECK_GT(pageSize_, reservedHeaderSize_); pageDataSize_ = pageSize_ - reservedHeaderSize_; //@todo reintroduce initialSize - need to develop easy way of @@ -76,7 +76,7 @@ FileBuffer::FileBuffer(FileMgr* fm, , pageSize_(pageSize) , chunkKey_(chunkKey) { CHECK(fm_); - calcHeaderBuffer(); + setBufferHeaderSize(); pageDataSize_ = pageSize_ - reservedHeaderSize_; } @@ -93,7 +93,7 @@ FileBuffer::FileBuffer(FileMgr* fm, // We are being assigned an existing FileBuffer on disk CHECK(fm_); - calcHeaderBuffer(); + setBufferHeaderSize(); int32_t lastPageId = -1; int32_t curPageId = 0; for (auto vecIt = headerStartIt; vecIt != headerEndIt; ++vecIt) { @@ -142,14 +142,25 @@ void FileBuffer::reserve(const size_t numBytes) { } } -void FileBuffer::calcHeaderBuffer() { - // 3 * sizeof(int32_t) is for headerSize, for pageId and versionEpoch - // sizeof(size_t) is for chunkSize - reservedHeaderSize_ = (chunkKey_.size() + 3) * sizeof(int32_t); - size_t headerMod = reservedHeaderSize_ % headerBufferOffset_; - if (headerMod > 0) { - reservedHeaderSize_ += headerBufferOffset_ - headerMod; +namespace { +size_t calculate_buffer_header_size(size_t chunk_size) { + // Additional 3 * sizeof(int32_t) is for headerSize, pageId, and versionEpoch + size_t header_size = (chunk_size + 3) * sizeof(int32_t); + size_t header_mod = header_size % FileBuffer::kHeaderBufferOffset; + if (header_mod > 0) { + header_size += FileBuffer::kHeaderBufferOffset - header_mod; } + return header_size; +} +} // namespace + +void FileBuffer::setBufferHeaderSize() { + reservedHeaderSize_ = calculate_buffer_header_size(chunkKey_.size()); +} + +size_t FileBuffer::getMinPageSize() { + constexpr size_t max_chunk_size{5}; + return calculate_buffer_header_size(max_chunk_size) + 1; } void FileBuffer::freePage(const Page& page) { diff --git a/DataMgr/FileMgr/FileBuffer.h b/DataMgr/FileMgr/FileBuffer.h index e82c78ea43..b066024e54 100644 --- a/DataMgr/FileMgr/FileBuffer.h +++ b/DataMgr/FileMgr/FileBuffer.h @@ -159,10 +159,12 @@ class FileBuffer : public AbstractBuffer { size_t numChunkPages() const; std::string dump() const; + static size_t getMinPageSize(); + // Used for testing void freePage(const Page& page); - static constexpr size_t headerBufferOffset_ = 32; + static constexpr size_t kHeaderBufferOffset{32}; private: // FileBuffer(const FileBuffer&); // private copy constructor @@ -178,7 +180,7 @@ class FileBuffer : public AbstractBuffer { const bool writeMetadata = false); void writeMetadata(const int32_t epoch); void readMetadata(const Page& page); - void calcHeaderBuffer(); + void setBufferHeaderSize(); void freePage(const Page& page, const bool isRolloff); void freePagesBeforeEpochForMultiPage(MultiPage& multiPage, diff --git a/Parser/ParserNode.cpp b/Parser/ParserNode.cpp index 0865ee492a..59ca72f5e3 100644 --- a/Parser/ParserNode.cpp +++ b/Parser/ParserNode.cpp @@ -46,6 +46,7 @@ #include "Catalog/Catalog.h" #include "Catalog/DataframeTableDescriptor.h" #include "Catalog/SharedDictionaryValidator.h" +#include "DataMgr/FileMgr/FileBuffer.h" #include "Fragmenter/InsertOrderFragmenter.h" #include "Fragmenter/SortedOrderFragmenter.h" #include "Fragmenter/TargetValueConvertersFactories.h" @@ -2933,8 +2934,14 @@ decltype(auto) get_header_def(DataframeTableDescriptor& df_td, decltype(auto) get_page_size_def(TableDescriptor& td, const NameValueAssign* p, const std::list& columns) { - return get_property_value(p, - [&td](const auto val) { td.fragPageSize = val; }); + return get_property_value(p, [&td](const auto val) { + const auto min_page_size = File_Namespace::FileBuffer::getMinPageSize(); + if (val < min_page_size) { + throw std::runtime_error("page_size cannot be less than " + + std::to_string(min_page_size)); + } + td.fragPageSize = val; + }); } decltype(auto) get_max_rows_def(TableDescriptor& td, const NameValueAssign* p, diff --git a/Parser/ParserNode.h b/Parser/ParserNode.h index ad1434153f..f5889f75b6 100644 --- a/Parser/ParserNode.h +++ b/Parser/ParserNode.h @@ -2240,11 +2240,24 @@ struct PositiveOrZeroValidate { } }; +namespace { +template +const std::string* validate_and_get_str(T name_value_assign) { + auto str = dynamic_cast(name_value_assign->get_value()); + if (!str) { + auto option_name = name_value_assign->get_name(); + CHECK(option_name); + throw std::runtime_error("The \"" + *option_name + "\" option must be a string."); + } + return str->get_stringval(); +} +} // namespace + template <> struct DefaultValidate { template decltype(auto) operator()(T t) { - const auto val = static_cast(t->get_value())->get_stringval(); + const auto val = validate_and_get_str(t); CHECK(val); const auto val_upper = boost::to_upper_copy(*val); return val_upper; @@ -2254,7 +2267,7 @@ struct DefaultValidate { struct CaseSensitiveValidate { template decltype(auto) operator()(T t) { - const auto val = static_cast(t->get_value())->get_stringval(); + const auto val = validate_and_get_str(t); CHECK(val); return *val; } diff --git a/Tests/CachingFileMgrTest.cpp b/Tests/CachingFileMgrTest.cpp index 6cfd6bf5bf..92dd1a48e3 100644 --- a/Tests/CachingFileMgrTest.cpp +++ b/Tests/CachingFileMgrTest.cpp @@ -55,7 +55,7 @@ class CachingFileMgrTest : public testing::Test { // Keep page size small for these tests so we can hit limits more easily. static constexpr size_t page_size_ = 64; static constexpr size_t page_data_size_ = - page_size_ - fn::FileBuffer::headerBufferOffset_; + page_size_ - fn::FileBuffer::kHeaderBufferOffset; static constexpr size_t data_file_size_ = page_size_ * fn::CachingFileMgr::DEFAULT_NUM_PAGES_PER_DATA_FILE; static constexpr size_t meta_file_size_ = diff --git a/Tests/CreateAndDropTableDdlTest.cpp b/Tests/CreateAndDropTableDdlTest.cpp index 43d48ac4e8..5b0b64ca7a 100644 --- a/Tests/CreateAndDropTableDdlTest.cpp +++ b/Tests/CreateAndDropTableDdlTest.cpp @@ -27,6 +27,7 @@ #include "Catalog/ForeignTable.h" #include "Catalog/TableDescriptor.h" #include "DBHandlerTestHelpers.h" +#include "DataMgr/FileMgr/FileBuffer.h" #include "Fragmenter/FragmentDefaultValues.h" #include "Shared/SysDefinitions.h" #include "Shared/scope.h" @@ -75,7 +76,7 @@ class CreateAndDropTableDdlTest : public DBHandlerTestFixture { std::string getCreateTableQuery(const ddl_utils::TableType table_type, const std::string& table_name, const std::string& columns, - bool if_not_exists = false) { + bool if_not_exists = false) const { return getCreateTableQuery(table_type, table_name, columns, {}, if_not_exists); } @@ -83,7 +84,7 @@ class CreateAndDropTableDdlTest : public DBHandlerTestFixture { const std::string& table_name, const std::string& columns, std::map options, - bool if_not_exists = false) { + bool if_not_exists = false) const { std::string query{"CREATE "}; if (table_type == ddl_utils::TableType::FOREIGN_TABLE) { query += "FOREIGN TABLE "; @@ -132,7 +133,7 @@ class CreateAndDropTableDdlTest : public DBHandlerTestFixture { return query; } - std::string getTestFilePath() { + std::string getTestFilePath() const { return bf::canonical("../../Tests/FsiDataFiles/example_1.csv").string(); } @@ -2438,6 +2439,85 @@ TEST_F(CommentsBeforeCommandTest, MultiLineCommentBeforeCommand) { "SQL statements starting with comments are currently not allowed."); } +class TableOptionsValidationTest : public CreateAndDropTableDdlTest { + protected: + void TearDown() override { + sql(getDropTableQuery(ddl_utils::TableType::TABLE, "test_table", true)); + CreateAndDropTableDdlTest::TearDown(); + } + + std::string getCreateTableQuery( + const std::map& options) const { + return CreateAndDropTableDdlTest::getCreateTableQuery( + ddl_utils::TableType::TABLE, "test_table", "(i INTEGER)", options); + } +}; + +class PageSizeValidationTest : public TableOptionsValidationTest { + protected: + std::string getCreateTableQuery(size_t page_size) const { + return TableOptionsValidationTest::getCreateTableQuery( + {{"page_size", std::to_string(page_size)}}); + } +}; + +TEST_F(PageSizeValidationTest, BelowMinPageSize) { + const auto min_page_size = File_Namespace::FileBuffer::getMinPageSize(); + queryAndAssertException( + PageSizeValidationTest::getCreateTableQuery(min_page_size - 1), + "page_size cannot be less than " + std::to_string(min_page_size)); +} + +TEST_F(PageSizeValidationTest, MinPageSize) { + sql(PageSizeValidationTest::getCreateTableQuery( + File_Namespace::FileBuffer::getMinPageSize())); + sql("INSERT INTO test_table VALUES (1);"); + sqlAndCompareResult("SELECT * FROM test_table;", {{i(1)}}); +} + +class StringTypeErrorValidationTest : public TableOptionsValidationTest, + public testing::WithParamInterface {}; + +TEST_P(StringTypeErrorValidationTest, StringOptions) { + const auto& option_name = GetParam(); + queryAndAssertException(getCreateTableQuery({{option_name, "0"}}), + "The \"" + option_name + "\" option must be a string."); +} + +INSTANTIATE_TEST_SUITE_P( + OptionTypeValidation, + StringTypeErrorValidationTest, + ::testing::Values("partitions", "sort_column", "storage_type", "vacuum")); + +class IntTypeErrorValidationTest : public TableOptionsValidationTest, + public testing::WithParamInterface { + protected: + std::string getCreateTableQuery(const std::map& options, + bool include_shard_key) const { + return CreateAndDropTableDdlTest::getCreateTableQuery( + ddl_utils::TableType::TABLE, + "test_table", + "(i INTEGER"s + (include_shard_key ? ", SHARD KEY (i)" : "") + ")", + options); + } +}; + +TEST_P(IntTypeErrorValidationTest, IntegerOptions) { + const auto& option_name = GetParam(); + queryAndAssertException( + getCreateTableQuery({{option_name, "'test'"}}, option_name == "shard_count"), + to_upper(option_name) + " must be an integer literal."); +} + +INSTANTIATE_TEST_SUITE_P(OptionTypeValidation, + IntTypeErrorValidationTest, + ::testing::Values("fragment_size", + "max_chunk_size", + "page_size", + "max_rows", + "shard_count", + "max_rollback_epochs")); + int main(int argc, char** argv) { g_enable_fsi = true; TestHelpers::init_logger_stderr_only(argc, argv); From e478aca965c00985e6b1891428b2d0d7894f301e Mon Sep 17 00:00:00 2001 From: yoonminnam <53632385+yoonminnam@users.noreply.github.com> Date: Tue, 26 Sep 2023 22:57:05 -0400 Subject: [PATCH 09/72] Missing locking when computing StringDictionary's cache size (#7540) * Add missing lock * Add test * Add string column query to check the concurrency under TSAN Signed-off-by: Misiu Godfrey --- StringDictionary/StringDictionary.cpp | 1 + Tests/ParallelExecutorsTest.cpp | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/StringDictionary/StringDictionary.cpp b/StringDictionary/StringDictionary.cpp index 263e7766d3..5d80bd3919 100644 --- a/StringDictionary/StringDictionary.cpp +++ b/StringDictionary/StringDictionary.cpp @@ -2081,6 +2081,7 @@ void translate_string_ids(std::vector& dest_ids, } size_t StringDictionary::computeCacheSize() const { + std::shared_lock read_lock(rw_mutex_); return string_id_string_dict_hash_table_.size() * sizeof(int32_t) + hash_cache_.size() * sizeof(string_dict_hash_t) + sorted_cache.size() * sizeof(int32_t) + like_cache_size_ + regex_cache_size_ + diff --git a/Tests/ParallelExecutorsTest.cpp b/Tests/ParallelExecutorsTest.cpp index 03e9e41b70..34bddf404b 100644 --- a/Tests/ParallelExecutorsTest.cpp +++ b/Tests/ParallelExecutorsTest.cpp @@ -142,6 +142,13 @@ class BaseTestFixture : public DBHandlerTestFixture, check_returned_rows("SELECT d, f, COUNT(*) FROM " + table_name + " GROUP BY d, f ORDER BY f DESC NULLS LAST LIMIT 5;", 5); + sqlAndCompareResult("SELECT COUNT(*) FROM " + table_name + " WHERE str like 'hello';", + {{i(10)}}); + sqlAndCompareResult( + "SELECT COUNT(*) FROM " + table_name + " WHERE str ilike 'hello';", {{i(10)}}); + sqlAndCompareResult( + "SELECT COUNT(*) FROM " + table_name + " WHERE str REGEXP '^[a-z]+r$';", + {{i(0)}}); check_returned_rows( "SELECT approx_count_distinct(d), approx_count_distinct(str), i64, i32, " "i16 FROM " + From dc3a45d4ddb9670c493381d6d0152c8f5c61d3d5 Mon Sep 17 00:00:00 2001 From: yoonminnam <53632385+yoonminnam@users.noreply.github.com> Date: Wed, 27 Sep 2023 19:16:31 -0400 Subject: [PATCH 10/72] Improve the performance of `LIKE` operator on dictionary-encoded string column (#7530) * Avoid redundant i64 vector copy * Parallel resultset copy * Address comments * Templatize getLike function * Refactor is_like impl selection logic * Refactor parallelization logic Signed-off-by: Misiu Godfrey --- QueryEngine/NativeCodegen.cpp | 8 +- QueryEngine/StringOpsIR.cpp | 21 +-- StringDictionary/StringDictionary.cpp | 160 ++++++++++++--------- StringDictionary/StringDictionary.h | 22 ++- StringDictionary/StringDictionaryClient.h | 22 ++- StringDictionary/StringDictionaryProxy.cpp | 55 +++---- StringDictionary/StringDictionaryProxy.h | 9 +- Utils/StringLike.cpp | 12 +- Utils/StringLike.h | 6 +- 9 files changed, 183 insertions(+), 132 deletions(-) diff --git a/QueryEngine/NativeCodegen.cpp b/QueryEngine/NativeCodegen.cpp index 76c4334653..41cb4494c8 100644 --- a/QueryEngine/NativeCodegen.cpp +++ b/QueryEngine/NativeCodegen.cpp @@ -838,10 +838,10 @@ declare i1 @string_like(i8*, i32, i8*, i32, i8); declare i1 @string_ilike(i8*, i32, i8*, i32, i8); declare i8 @string_like_nullable(i8*, i32, i8*, i32, i8, i8); declare i8 @string_ilike_nullable(i8*, i32, i8*, i32, i8, i8); -declare i1 @string_like_simple(i8*, i32, i8*, i32); -declare i1 @string_ilike_simple(i8*, i32, i8*, i32); -declare i8 @string_like_simple_nullable(i8*, i32, i8*, i32, i8); -declare i8 @string_ilike_simple_nullable(i8*, i32, i8*, i32, i8); +declare i1 @string_like_simple(i8*, i32, i8*, i32, i8); +declare i1 @string_ilike_simple(i8*, i32, i8*, i32, i8); +declare i8 @string_like_simple_nullable(i8*, i32, i8*, i32, i8, i8); +declare i8 @string_ilike_simple_nullable(i8*, i32, i8*, i32, i8, i8); declare i1 @string_lt(i8*, i32, i8*, i32); declare i1 @string_le(i8*, i32, i8*, i32); declare i1 @string_gt(i8*, i32, i8*, i32); diff --git a/QueryEngine/StringOpsIR.cpp b/QueryEngine/StringOpsIR.cpp index a58b4a19b9..20bc465559 100644 --- a/QueryEngine/StringOpsIR.cpp +++ b/QueryEngine/StringOpsIR.cpp @@ -631,13 +631,14 @@ llvm::Value* CodeGenerator::codegen(const Analyzer::LikeExpr* expr, auto like_expr_arg_lvs = codegen(expr->get_like_expr(), true, co); CHECK_EQ(size_t(3), like_expr_arg_lvs.size()); const bool is_nullable{!expr->get_arg()->get_type_info().get_notnull()}; - std::vector str_like_args{ - str_lv[1], str_lv[2], like_expr_arg_lvs[1], like_expr_arg_lvs[2]}; + std::vector str_like_args{str_lv[1], + str_lv[2], + like_expr_arg_lvs[1], + like_expr_arg_lvs[2], + cgen_state_->llInt(int8_t(escape_char))}; std::string fn_name{expr->get_is_ilike() ? "string_ilike" : "string_like"}; if (expr->get_is_simple()) { fn_name += "_simple"; - } else { - str_like_args.push_back(cgen_state_->llInt(int8_t(escape_char))); } if (is_nullable) { fn_name += "_nullable"; @@ -714,12 +715,14 @@ llvm::Value* CodeGenerator::codegenDictLike( CHECK_EQ(kENCODING_NONE, pattern_ti.get_compression()); const auto& pattern_datum = pattern->get_constval(); const auto& pattern_str = *pattern_datum.stringval; - const auto matching_ids = sdp->getLike(pattern_str, ilike, is_simple, escape_char); - // InIntegerSet requires 64-bit values - std::vector matching_ids_64(matching_ids.size()); - std::copy(matching_ids.begin(), matching_ids.end(), matching_ids_64.begin()); + auto work_timer = timer_start(); + const auto matching_ids = + sdp->getLike(pattern_str, ilike, is_simple, escape_char); + auto const work_ms = timer_stop(work_timer); + VLOG(3) << "Processing like operator with the pattern " << pattern_str << " took " + << work_ms << " ms (# matching elems: " << matching_ids.size() << ")"; const auto in_values = std::make_shared( - dict_like_arg, matching_ids_64, dict_like_arg_ti.get_notnull()); + dict_like_arg, matching_ids, dict_like_arg_ti.get_notnull()); return codegen(in_values.get(), co); } diff --git a/StringDictionary/StringDictionary.cpp b/StringDictionary/StringDictionary.cpp index 5d80bd3919..afdf53a8f5 100644 --- a/StringDictionary/StringDictionary.cpp +++ b/StringDictionary/StringDictionary.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -49,6 +50,7 @@ #include "Utils/StringLike.h" #include "LeafHostInfo.h" +#include "Shared/measure.h" bool g_cache_string_hash{true}; @@ -812,80 +814,103 @@ size_t StringDictionary::storageEntryCount() const { return str_count_; } -namespace { - -bool is_like(const std::string& str, - const std::string& pattern, - const bool icase, - const bool is_simple, - const char escape) { - return icase - ? (is_simple ? string_ilike_simple( - str.c_str(), str.size(), pattern.c_str(), pattern.size()) - : string_ilike(str.c_str(), - str.size(), - pattern.c_str(), - pattern.size(), - escape)) - : (is_simple ? string_like_simple( - str.c_str(), str.size(), pattern.c_str(), pattern.size()) - : string_like(str.c_str(), - str.size(), - pattern.c_str(), - pattern.size(), - escape)); +template +std::vector StringDictionary::getLikeImpl(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape, + const size_t generation) const { + constexpr size_t grain_size{1000}; + auto is_like_impl = icase ? is_simple ? string_ilike_simple : string_ilike + : is_simple ? string_like_simple + : string_like; + auto const num_threads = static_cast(cpu_threads()); + std::vector> worker_results(num_threads); + tbb::task_arena limited_arena(num_threads); + limited_arena.execute([&] { + tbb::parallel_for( + tbb::blocked_range(0, generation, grain_size), + [&is_like_impl, &pattern, &escape, &worker_results, this]( + const tbb::blocked_range& range) { + auto& result_vector = + worker_results[tbb::this_task_arena::current_thread_index()]; + for (size_t i = range.begin(); i < range.end(); ++i) { + const auto str = getStringUnlocked(i); + if (is_like_impl( + str.c_str(), str.size(), pattern.c_str(), pattern.size(), escape)) { + result_vector.push_back(i); + } + } + }); + }); + // partial_sum to get 1) a start offset for each thread and 2) the total # elems + std::vector start_offsets(num_threads + 1, 0); + auto vec_size = [](std::vector const& vec) { return vec.size(); }; + auto begin = boost::make_transform_iterator(worker_results.begin(), vec_size); + auto end = boost::make_transform_iterator(worker_results.end(), vec_size); + std::partial_sum(begin, end, start_offsets.begin() + 1); // first element is 0 + + std::vector result(start_offsets[num_threads]); + limited_arena.execute([&] { + tbb::parallel_for( + tbb::blocked_range(0, num_threads, 1), + [&worker_results, &result, &start_offsets]( + const tbb::blocked_range& range) { + auto& result_vector = worker_results[range.begin()]; + auto const start_offset = start_offsets[range.begin()]; + std::copy( + result_vector.begin(), result_vector.end(), result.begin() + start_offset); + }, + tbb::static_partitioner()); + }); + return result; } - -} // namespace - -std::vector StringDictionary::getLike(const std::string& pattern, - const bool icase, - const bool is_simple, - const char escape, - const size_t generation) const { +template <> +std::vector StringDictionary::getLike(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape, + const size_t generation) const { std::lock_guard write_lock(rw_mutex_); if (isClient()) { - return client_->get_like(pattern, icase, is_simple, escape, generation); + return client_->get_like_i32(pattern, icase, is_simple, escape, generation); } const auto cache_key = std::make_tuple(pattern, icase, is_simple, escape); - const auto it = like_cache_.find(cache_key); - if (it != like_cache_.end()) { + const auto it = like_i32_cache_.find(cache_key); + if (it != like_i32_cache_.end()) { return it->second; } - std::vector result; - std::vector workers; - int worker_count = cpu_threads(); - CHECK_GT(worker_count, 0); - std::vector> worker_results(worker_count); - CHECK_LE(generation, str_count_); - for (int worker_idx = 0; worker_idx < worker_count; ++worker_idx) { - workers.emplace_back([&worker_results, - &pattern, - generation, - icase, - is_simple, - escape, - worker_idx, - worker_count, - this]() { - for (size_t string_id = worker_idx; string_id < generation; - string_id += worker_count) { - const auto str = getStringUnlocked(string_id); - if (is_like(str, pattern, icase, is_simple, escape)) { - worker_results[worker_idx].push_back(string_id); - } - } - }); - } - for (auto& worker : workers) { - worker.join(); + + auto result = getLikeImpl(pattern, icase, is_simple, escape, generation); + // place result into cache for reuse if similar query + const auto it_ok = like_i32_cache_.insert(std::make_pair(cache_key, result)); + like_cache_size_ += (pattern.size() + 3 + (result.size() * sizeof(int32_t))); + + CHECK(it_ok.second); + + return result; +} + +template <> +std::vector StringDictionary::getLike(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape, + const size_t generation) const { + std::lock_guard write_lock(rw_mutex_); + if (isClient()) { + return client_->get_like_i64(pattern, icase, is_simple, escape, generation); } - for (const auto& worker_result : worker_results) { - result.insert(result.end(), worker_result.begin(), worker_result.end()); + const auto cache_key = std::make_tuple(pattern, icase, is_simple, escape); + const auto it = like_i64_cache_.find(cache_key); + if (it != like_i64_cache_.end()) { + return it->second; } + + auto result = getLikeImpl(pattern, icase, is_simple, escape, generation); // place result into cache for reuse if similar query - const auto it_ok = like_cache_.insert(std::make_pair(cache_key, result)); - like_cache_size_ += (pattern.size() + 3 + (result.size() * sizeof(int32_t))); + const auto it_ok = like_i64_cache_.insert(std::make_pair(cache_key, result)); + like_cache_size_ += (pattern.size() + 3 + (result.size() * sizeof(int64_t))); CHECK(it_ok.second); @@ -1599,8 +1624,11 @@ void* StringDictionary::addMemoryCapacity(void* addr, } void StringDictionary::invalidateInvertedIndex() noexcept { - if (!like_cache_.empty()) { - decltype(like_cache_)().swap(like_cache_); + if (!like_i32_cache_.empty()) { + decltype(like_i32_cache_)().swap(like_i32_cache_); + } + if (!like_i64_cache_.empty()) { + decltype(like_i64_cache_)().swap(like_i64_cache_); } if (!regex_cache_.empty()) { decltype(regex_cache_)().swap(regex_cache_); diff --git a/StringDictionary/StringDictionary.h b/StringDictionary/StringDictionary.h index e6e53763d3..99c6873b19 100644 --- a/StringDictionary/StringDictionary.h +++ b/StringDictionary/StringDictionary.h @@ -99,11 +99,19 @@ class StringDictionary { std::pair getStringBytes(int32_t string_id) const noexcept; size_t storageEntryCount() const; - std::vector getLike(const std::string& pattern, - const bool icase, - const bool is_simple, - const char escape, - const size_t generation) const; + template + std::vector getLike(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape, + const size_t generation) const; + + template + std::vector getLikeImpl(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape, + const size_t generation) const; std::vector getCompare(const std::string& pattern, const std::string& comp_operator, @@ -284,7 +292,9 @@ class StringDictionary { size_t payload_file_off_; mutable std::shared_mutex rw_mutex_; mutable std::map, std::vector> - like_cache_; + like_i32_cache_; + mutable std::map, std::vector> + like_i64_cache_; mutable size_t like_cache_size_; mutable std::map, std::vector> regex_cache_; mutable size_t regex_cache_size_; diff --git a/StringDictionary/StringDictionaryClient.h b/StringDictionary/StringDictionaryClient.h index 13c3846beb..e2edfe1429 100644 --- a/StringDictionary/StringDictionaryClient.h +++ b/StringDictionary/StringDictionaryClient.h @@ -47,14 +47,24 @@ class StringDictionaryClient { return 0; }; - std::vector get_like(const std::string& pattern, - const bool icase, - const bool is_simple, - const char escape, - const int64_t generation) { + std::vector get_like_i32(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape, + const int64_t generation) { CHECK(false); return std::vector{}; - }; + } + + std::vector get_like_i64(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape, + const int64_t generation) { + CHECK(false); + return std::vector{}; + } + std::vector get_compare(const std::string& pattern, const std::string& comp_operator, diff --git a/StringDictionary/StringDictionaryProxy.cpp b/StringDictionary/StringDictionaryProxy.cpp index c6e52a30c4..32c9e34f89 100644 --- a/StringDictionary/StringDictionaryProxy.cpp +++ b/StringDictionary/StringDictionaryProxy.cpp @@ -465,46 +465,37 @@ StringDictionaryProxy::IdMap StringDictionaryProxy::buildUnionTranslationMapToOt return id_map; } -namespace { - -bool is_like(const std::string& str, - const std::string& pattern, - const bool icase, - const bool is_simple, - const char escape) { - return icase - ? (is_simple ? string_ilike_simple( - str.c_str(), str.size(), pattern.c_str(), pattern.size()) - : string_ilike(str.c_str(), - str.size(), - pattern.c_str(), - pattern.size(), - escape)) - : (is_simple ? string_like_simple( - str.c_str(), str.size(), pattern.c_str(), pattern.size()) - : string_like(str.c_str(), - str.size(), - pattern.c_str(), - pattern.size(), - escape)); -} - -} // namespace - -std::vector StringDictionaryProxy::getLike(const std::string& pattern, - const bool icase, - const bool is_simple, - const char escape) const { +template +std::vector StringDictionaryProxy::getLike(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape) const { CHECK_GE(generation_, 0); - auto result = string_dict_->getLike(pattern, icase, is_simple, escape, generation_); + auto result = string_dict_->getLike(pattern, icase, is_simple, escape, generation_); + auto is_like_impl = icase ? is_simple ? string_ilike_simple : string_ilike + : is_simple ? string_like_simple + : string_like; for (unsigned index = 0; index < transient_string_vec_.size(); ++index) { - if (is_like(*transient_string_vec_[index], pattern, icase, is_simple, escape)) { + auto const str = *transient_string_vec_[index]; + if (is_like_impl(str.c_str(), str.size(), pattern.c_str(), pattern.size(), escape)) { result.push_back(transientIndexToId(index)); } } return result; } +template std::vector StringDictionaryProxy::getLike( + const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape) const; + +template std::vector StringDictionaryProxy::getLike( + const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape) const; + namespace { bool do_compare(const std::string& str, diff --git a/StringDictionary/StringDictionaryProxy.h b/StringDictionary/StringDictionaryProxy.h index 102c72c489..633bf9afde 100644 --- a/StringDictionary/StringDictionaryProxy.h +++ b/StringDictionary/StringDictionaryProxy.h @@ -207,10 +207,11 @@ class StringDictionaryProxy { void updateGeneration(const int64_t generation) noexcept; - std::vector getLike(const std::string& pattern, - const bool icase, - const bool is_simple, - const char escape) const; + template + std::vector getLike(const std::string& pattern, + const bool icase, + const bool is_simple, + const char escape) const; std::vector getCompare(const std::string& pattern, const std::string& comp_operator) const; diff --git a/Utils/StringLike.cpp b/Utils/StringLike.cpp index 449c34cb90..a3ee12fe83 100644 --- a/Utils/StringLike.cpp +++ b/Utils/StringLike.cpp @@ -38,10 +38,13 @@ DEVICE static int inline lowercase(char c) { return c; } +// escape_char does nothing, it's a placeholder to fit # arguments for both +// string_like and string_like_simple functions extern "C" RUNTIME_EXPORT DEVICE bool string_like_simple(const char* str, const int32_t str_len, const char* pattern, - const int32_t pat_len) { + const int32_t pat_len, + char escape_char) { int i, j; int search_len = str_len - pat_len + 1; for (i = 0; i < search_len; ++i) { @@ -54,10 +57,12 @@ extern "C" RUNTIME_EXPORT DEVICE bool string_like_simple(const char* str, return false; } +// escape_char does nothing and it is intentional as describe above extern "C" RUNTIME_EXPORT DEVICE bool string_ilike_simple(const char* str, const int32_t str_len, const char* pattern, - const int32_t pat_len) { + const int32_t pat_len, + char escape_char) { int i, j; int search_len = str_len - pat_len + 1; for (i = 0; i < search_len; ++i) { @@ -75,11 +80,12 @@ extern "C" RUNTIME_EXPORT DEVICE bool string_ilike_simple(const char* str, const int32_t lhs_len, \ const char* rhs, \ const int32_t rhs_len, \ + char escape_char, \ const int8_t bool_null) { \ if (!lhs || !rhs) { \ return bool_null; \ } \ - return base_func(lhs, lhs_len, rhs, rhs_len) ? 1 : 0; \ + return base_func(lhs, lhs_len, rhs, rhs_len, escape_char) ? 1 : 0; \ } STR_LIKE_SIMPLE_NULLABLE(string_like_simple) diff --git a/Utils/StringLike.h b/Utils/StringLike.h index ffa231cf1f..9342d70f5d 100644 --- a/Utils/StringLike.h +++ b/Utils/StringLike.h @@ -55,12 +55,14 @@ extern "C" RUNTIME_EXPORT DEVICE bool string_ilike(const char* str, extern "C" RUNTIME_EXPORT DEVICE bool string_like_simple(const char* str, const int32_t str_len, const char* pattern, - const int32_t pat_len); + const int32_t pat_len, + char escape_char); extern "C" RUNTIME_EXPORT DEVICE bool string_ilike_simple(const char* str, const int32_t str_len, const char* pattern, - const int32_t pat_len); + const int32_t pat_len, + char escape_char); extern "C" RUNTIME_EXPORT DEVICE bool string_lt(const char* lhs, const int32_t lhs_len, From b561820b5bd061c43b7b538e12d9c68397c7db89 Mon Sep 17 00:00:00 2001 From: Andrew Do <34977845+AndrewVDo@users.noreply.github.com> Date: Wed, 27 Sep 2023 21:04:22 -0700 Subject: [PATCH 11/72] Disallow "CREATE OR REPLACE" DDL (#7195) * Disables `CREATE OR REPLACE` for all database objects except `MODEL` Co-authored-by: Matt Gara Signed-off-by: Misiu Godfrey --- Tests/ArrowCsvForeignStorageTest.cpp | 32 ++++++++ Tests/CreateAndDropTableDdlTest.cpp | 62 +++++++++++++- Tests/CtasUpdateTest.cpp | 14 ++++ Tests/DBObjectPrivilegesTest.cpp | 82 ++++++++++++++++++- Tests/ForeignServerDdlTest.cpp | 11 +++ java/calcite/src/main/codegen/config.fmpp | 24 +++--- .../src/main/codegen/includes/ddlParser.ftl | 61 +++++++++++--- .../codegen/includes/foreignServerParser.ftl | 2 +- .../codegen/includes/foreignTableParser.ftl | 2 +- .../codegen/includes/userMappingParser.ftl | 2 +- .../parser/extension/ddl/SqlCreateTable.java | 4 +- .../parser/extension/ddl/SqlCreateView.java | 11 +-- .../parser/extension/ddl/SqlDdlNodes.java | 6 +- 13 files changed, 264 insertions(+), 49 deletions(-) diff --git a/Tests/ArrowCsvForeignStorageTest.cpp b/Tests/ArrowCsvForeignStorageTest.cpp index fc5bf88ec0..ca7778f236 100644 --- a/Tests/ArrowCsvForeignStorageTest.cpp +++ b/Tests/ArrowCsvForeignStorageTest.cpp @@ -559,6 +559,38 @@ TEST_F(BooleanTest, CheckWithoutNulls) { {1, 1, 1, 0, 0, 1, 0, 1}); } +class CreateDataframeTest : public ::testing::Test { + protected: + void SetUp() override { + ASSERT_NO_THROW(run_ddl_statement("drop table if exists test_dataframe;")); + } + + void TearDown() override { + ASSERT_NO_THROW(run_ddl_statement("drop table if exists test_dataframe;")); + } + + void queryAndAssertPartialException(const std::string& query, + const std::string& error_message) { + try { + run_ddl_statement(query); + FAIL() << "An exception should have been thrown for this test case"; + } catch (const std::exception& e) { + std::string exception_message{e.what()}; + ASSERT_TRUE(exception_message.find(error_message) != std::string::npos) + << "Exception message: " << exception_message + << ", expected partial error message: " << error_message; + } + } +}; +TEST_F(CreateDataframeTest, CreateOrReplaceDataframe) { + queryAndAssertPartialException( + "CREATE OR REPLACE DATAFRAME test_dataframe(idx " + "integer) FROM 'CSV:../../Tests/FsiDataFiles/0.csv';", + R"(SQL Error: Encountered "DATAFRAME" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + } // namespace int main(int argc, char** argv) { diff --git a/Tests/CreateAndDropTableDdlTest.cpp b/Tests/CreateAndDropTableDdlTest.cpp index 5b0b64ca7a..372fe20e77 100644 --- a/Tests/CreateAndDropTableDdlTest.cpp +++ b/Tests/CreateAndDropTableDdlTest.cpp @@ -76,16 +76,27 @@ class CreateAndDropTableDdlTest : public DBHandlerTestFixture { std::string getCreateTableQuery(const ddl_utils::TableType table_type, const std::string& table_name, const std::string& columns, - bool if_not_exists = false) const { - return getCreateTableQuery(table_type, table_name, columns, {}, if_not_exists); + bool if_not_exists = false, + bool or_replace = false, + bool temporary_table = false) const { + return getCreateTableQuery( + table_type, table_name, columns, {}, if_not_exists, or_replace, temporary_table); } std::string getCreateTableQuery(const ddl_utils::TableType table_type, const std::string& table_name, const std::string& columns, std::map options, - bool if_not_exists = false) const { + bool if_not_exists = false, + bool or_replace = false, + bool temporary_table = false) const { std::string query{"CREATE "}; + if (or_replace) { + query += "OR REPLACE "; + } + if (temporary_table) { + query += "TEMPORARY "; + } if (table_type == ddl_utils::TableType::FOREIGN_TABLE) { query += "FOREIGN TABLE "; } else { @@ -167,12 +178,14 @@ class CreateTableTest : public CreateAndDropTableDdlTest, protected: void SetUp() override { CreateAndDropTableDdlTest::SetUp(); + sql("DROP VIEW IF EXISTS test_view"); sql(getDropTableQuery(GetParam(), "test_table", true)); dropTestUser(); } void TearDown() override { g_enable_fsi = true; + sql("DROP VIEW IF EXISTS test_view"); sql(getDropTableQuery(GetParam(), "test_table", true)); dropTestUser(); CreateAndDropTableDdlTest::TearDown(); @@ -1724,6 +1737,49 @@ TEST_P(CreateTableTest, RealAlias) { ASSERT_EQ(cd->columnType.get_type(), kFLOAT); } +TEST_P(CreateTableTest, CreateOrReplaceTable) { + auto query = + getCreateTableQuery(GetParam(), "test_table", "(idx INTEGER)", false, true); + // using a partial exception for the sake of brevity + if (GetParam() == ddl_utils::TableType::FOREIGN_TABLE) { + queryAndAssertPartialException( + query, + R"(SQL Error: Encountered "FOREIGN" at line 1, column 19. +Was expecting: + "MODEL" ...)"); + } else { + queryAndAssertPartialException(query, + R"(SQL Error: Encountered "TABLE" at line 1, column 19. +Was expecting: + "MODEL" ...)"); + } +} + +TEST_P(CreateTableTest, CreateOrReplaceTemporaryTable) { + if (GetParam() == ddl_utils::TableType::FOREIGN_TABLE) { + GTEST_SKIP() << "Foreign tables can't be temporary."; + } + auto query = + getCreateTableQuery(GetParam(), "test_table", "(idx INTEGER)", false, true, true); + // using a partial exception for the sake of brevity + queryAndAssertPartialException( + query, + R"(SQL Error: Encountered "TEMPORARY" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + +TEST_P(CreateTableTest, CreateOrReplaceView) { + sql(getCreateTableQuery(GetParam(), "test_table", "(idx INTEGER)")); + auto query = + std::string("CREATE OR REPLACE VIEW test_view AS SELECT * FROM test_table"); + // using a partial exception for the sake of brevity + queryAndAssertPartialException(query, + R"(SQL Error: Encountered "VIEW" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + INSTANTIATE_TEST_SUITE_P(CreateAndDropTableDdlTest, DropTableTest, testing::Values(ddl_utils::TableType::TABLE, diff --git a/Tests/CtasUpdateTest.cpp b/Tests/CtasUpdateTest.cpp index eeb0ee0fea..299fc1bb54 100644 --- a/Tests/CtasUpdateTest.cpp +++ b/Tests/CtasUpdateTest.cpp @@ -3621,6 +3621,7 @@ class CtasTableTest : public DBHandlerTestFixture, void SetUp() override { DBHandlerTestFixture::SetUp(); // Default connection string outside of thrift + ASSERT_NO_THROW(sql("drop table if exists test_table;")); ASSERT_NO_THROW(sql("drop table if exists ctas_test;")); ASSERT_NO_THROW(sql("drop table if exists ctas_test_empty;")); ASSERT_NO_THROW(sql("drop table if exists ctas_test_full;")); @@ -3628,6 +3629,7 @@ class CtasTableTest : public DBHandlerTestFixture, } void TearDown() override { + ASSERT_NO_THROW(sql("drop table if exists test_table;")); ASSERT_NO_THROW(sql("drop table if exists ctas_test;")); ASSERT_NO_THROW(sql("drop table if exists ctas_test_empty;")); ASSERT_NO_THROW(sql("drop table if exists ctas_test_full;")); @@ -3815,6 +3817,18 @@ TEST_F(CtasTableTest, CreateTableAsSelect) { // } } +TEST_F(CtasTableTest, CreateOrReplaceCtasTable) { + sql("CREATE TABLE test_table (idx integer)"); + sql("INSERT INTO test_table (idx) VALUES (1), (2), (3)"); + auto query = std::string( + "CREATE OR REPLACE TABLE test_table_as_select AS SELECT * FROM test_table"); + // using a partial exception for the sake of brevity + queryAndAssertPartialException(query, + R"(SQL Error: Encountered "TABLE" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + class NullTextArrayTest : public DBHandlerTestFixture { protected: void SetUp() override { diff --git a/Tests/DBObjectPrivilegesTest.cpp b/Tests/DBObjectPrivilegesTest.cpp index 3ea5af686a..3f1bb58644 100644 --- a/Tests/DBObjectPrivilegesTest.cpp +++ b/Tests/DBObjectPrivilegesTest.cpp @@ -4722,13 +4722,32 @@ TEST(SyncUserWithRemoteProvider, IS_SUPER) { ASSERT_EQ(u2->isSuper, false); } -class DropUserTest : public DBHandlerTestFixture {}; +class CreateDropUserTest : public DBHandlerTestFixture { + void SetUp() override { + DBHandlerTestFixture::SetUp(); + sql("drop user if exists test_user"); + } -TEST_F(DropUserTest, DropAdmin) { + void TearDown() override { + sql("drop user if exists test_user"); + DBHandlerTestFixture::TearDown(); + } +}; + +TEST_F(CreateDropUserTest, DropAdmin) { queryAndAssertException("DROP USER admin;", "Cannot drop user. User admin is required to exist."); } +TEST_F(CreateDropUserTest, CreateOrReplaceUser) { + auto query = std::string("CREATE OR REPLACE USER test_user"); + // using a partial exception for the sake of brevity + queryAndAssertPartialException(query, + R"(SQL Error: Encountered "USER" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + class CreateDropDatabaseTest : public DBHandlerTestFixture { protected: void SetUp() override { @@ -4791,6 +4810,16 @@ TEST_F(CreateDropDatabaseTest, LegacyOrphanedDB) { {"orphan_db", ""}}); } +TEST_F(CreateDropDatabaseTest, CreateOrReplaceDatabase) { + auto query = std::string("CREATE OR REPLACE DATABASE orphan_db"); + // using a partial exception for the sake of brevity + queryAndAssertPartialException( + query, + R"(SQL Error: Encountered "DATABASE" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + class DatabaseCaseSensitiveTest : public DBHandlerTestFixture { protected: static void SetUpTestSuite() { @@ -4933,6 +4962,55 @@ TEST_F(DatabaseCaseSensitiveTest, GetInternalTableDetailsForDatabase) { ASSERT_EQ(table_details.row_desc[1].col_name, "rowid"); } +class CreatePolicy : public DBHandlerTestFixture { + protected: + void SetUp() override { + DBHandlerTestFixture::SetUp(); + sql("DROP TABLE IF EXISTS test_table"); + sql("DROP USER IF EXISTS test_user"); + sql("CREATE TABLE test_table(idx INTEGER)"); + sql("CREATE USER test_user"); + } + + void TearDown() override { + sql("DROP TABLE IF EXISTS test_table"); + sql("DROP USER IF EXISTS test_user"); + DBHandlerTestFixture::TearDown(); + } +}; + +TEST_F(CreatePolicy, CreateOrReplacePolicy) { + auto query = std::string( + "CREATE OR REPLACE POLICY ON COLUMN test_table.idx TO test_user VALUES(0)"); + // using a partial exception for the sake of brevity + queryAndAssertPartialException(query, + R"(SQL Error: Encountered "POLICY" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + +class CreateRole : public DBHandlerTestFixture { + protected: + void SetUp() override { + DBHandlerTestFixture::SetUp(); + sql("DROP ROLE IF EXISTS test_role"); + } + + void TearDown() override { + sql("DROP ROLE IF EXISTS test_role"); + DBHandlerTestFixture::TearDown(); + } +}; + +TEST_F(CreateRole, CreateOrReplaceRole) { + auto query = std::string("CREATE OR REPLACE ROLE test_role"); + // using a partial exception for the sake of brevity + queryAndAssertPartialException(query, + R"(SQL Error: Encountered "ROLE" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + int main(int argc, char* argv[]) { testing::InitGoogleTest(&argc, argv); diff --git a/Tests/ForeignServerDdlTest.cpp b/Tests/ForeignServerDdlTest.cpp index 4f83e11b1c..e81f0aa629 100644 --- a/Tests/ForeignServerDdlTest.cpp +++ b/Tests/ForeignServerDdlTest.cpp @@ -184,6 +184,17 @@ TEST_F(CreateForeignServerTest, MissingWithClause) { queryAndAssertException(query, "Foreign server options must contain \"STORAGE_TYPE\"."); } +TEST_F(CreateForeignServerTest, CreateOrReplaceServer) { + std::string query{ + "CREATE OR REPLACE SERVER test_server FOREIGN DATA WRAPPER delimited_file " + "WITH (storage_type = 'LOCAL_FILE', base_path = '/test_path/');"}; + // using a partial exception for the sake of brevity + queryAndAssertPartialException(query, + R"(SQL Error: Encountered "SERVER" at line 1, column 19. +Was expecting: + "MODEL" ...)"); +} + class ReservedServerNamePrefixTest : public DBHandlerTestFixture, public ::testing::WithParamInterface {}; diff --git a/java/calcite/src/main/codegen/config.fmpp b/java/calcite/src/main/codegen/config.fmpp index 76042cbeb0..2f1772b2a1 100644 --- a/java/calcite/src/main/codegen/config.fmpp +++ b/java/calcite/src/main/codegen/config.fmpp @@ -569,8 +569,9 @@ data: { ] # List of methods for parsing custom SQL statements. - # Note that DROP commands are handled specially by SqlCustomDrop - # and SHOW commands are handled specially by SqlCustomShow because + # Note that CREATE commands are handled by SqlCustomCreate, + # DROP commands are handled by SqlCustomDrop and SHOW + # commands are handled specially by SqlCustomShow because # all of the statements listed here have LOOKAHEAD(2) which is # hardcoded in Calcite's Parser.jj source code. statementParserMethods: [ @@ -584,6 +585,7 @@ data: { "SqlRenameTable(span())" "SqlInsertIntoTable(span())" "SqlKillQuery(span())" + "SqlCustomCreate(span())" "SqlCustomDrop(span())" "SqlCustomShow(span())" "SqlGrant(span())" @@ -625,18 +627,16 @@ data: { # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls. # Each must accept arguments "(SqlParserPos pos, boolean replace)". + # ---------------------------------------------------- + # No longer used in cases where REPLACE is unsupported. + # + # This broke away from the default Calcite implementation because we do + # not allow the optional "OR REPLACE" clause which is allowed by + # Calcite's default implementation of create. + # See: SqlCustomCreate() + # ---------------------------------------------------- createStatementParserMethods: [ - "SqlCreateDataframe" - "SqlCreateDB" - "SqlCreateServer" - "SqlCreateForeignTable" - "SqlCreateUserMapping" - "SqlCreateTable" - "SqlCreateUser" - "SqlCreateView" "SqlCreateModel" - "SqlCreateRole" - "SqlCreatePolicy" ] # List of methods for parsing extensions to "DROP" calls. diff --git a/java/calcite/src/main/codegen/includes/ddlParser.ftl b/java/calcite/src/main/codegen/includes/ddlParser.ftl index 784eed5b7f..1e9141b3f8 100644 --- a/java/calcite/src/main/codegen/includes/ddlParser.ftl +++ b/java/calcite/src/main/codegen/includes/ddlParser.ftl @@ -569,7 +569,7 @@ boolean TemporaryOpt() : * * CREATE TABLE [ IF NOT EXISTS ] AS