From 4b19a0d83427aa4cfd98b103cbe55ed75050eedf Mon Sep 17 00:00:00 2001 From: William Malpica Date: Mon, 12 Apr 2021 11:12:05 -0500 Subject: [PATCH 01/18] WIP, barely starting to figure stuff out --- .../BatchWindowFunctionProcessing.cpp | 308 +++++++++++++++++- .../BatchWindowFunctionProcessing.h | 40 +++ 2 files changed, 343 insertions(+), 5 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index f1e9301a0..b259104c5 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -238,11 +238,6 @@ ral::execution::task_result ComputeWindowKernel::do_process(std::vector< std::un std::unique_ptr windowed_table = std::make_unique(std::move(cudf_table_window), output_names); - if (windowed_table) { - cudf::size_type num_rows = windowed_table->num_rows(); - std::size_t num_bytes = windowed_table->sizeInBytes(); - } - output->addToCache(std::move(windowed_table)); }catch(const rmm::bad_alloc& e){ return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; @@ -1004,5 +999,308 @@ In the future, when we can have CacheData's shared between nodes, then we can re // END OverlapAccumulatorKernel + + +// BEGIN ComputeWindowKernelUnbounded + +ComputeWindowKernelUnbounded::ComputeWindowKernelUnbounded(std::size_t kernel_id, const std::string & queryString, + std::shared_ptr context, + std::shared_ptr query_graph) + : kernel{kernel_id, queryString, context, kernel_type::ComputeWindowKernel} { + this->query_graph = query_graph; + std::tie(this->preceding_value, this->following_value) = get_bounds_from_window_expression(this->expression); + this->frame_type = get_frame_type_from_over_clause(this->expression); + + std::tie(this->column_indices_to_agg, this->type_aggs_as_str, this->agg_param_values) = + get_cols_to_apply_window_and_cols_to_apply_agg(this->expression); + std::tie(this->column_indices_partitioned, std::ignore) = ral::operators::get_vars_to_partition(this->expression); + std::tie(this->column_indices_ordered, std::ignore) = ral::operators::get_vars_to_orders(this->expression); + + // if the window function has no partitioning but does have order by and a bounded window, then we need to remove the overlaps that are present in the data + if (column_indices_partitioned.size() == 0 && column_indices_ordered.size() > 0 && this->preceding_value > 0 && this->following_value > 0){ + this->remove_overlap = true; + } else { + this->remove_overlap = false; + } +} + +// WSM LEFT OFF HERE: lets make a full matrix of support and implementation and figure out the gaps +// want to include what aggs support what clauses + +// TODO: Support for RANK() and DENSE_RANK() +std::unique_ptr ComputeWindowKernelUnbounded::compute_column_from_window_function( + cudf::table_view input_table_cudf_view, + cudf::column_view col_view_to_agg, + std::size_t pos, int & agg_param_count ) { + + std::unique_ptr window_aggregation; + + // we want firs get the type of aggregation + if (this->agg_param_values.size() > agg_param_count && is_lag_or_lead_aggregation(this->type_aggs_as_str[pos])) { + window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[agg_param_count]); + agg_param_count++; + } else if (is_last_value_window(this->type_aggs_as_str[pos])) { + window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], -1); + } else { + window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos]); + } + + // want all columns to be partitioned + std::vector columns_to_partition; + for (std::size_t col_i = 0; col_i < this->column_indices_partitioned.size(); ++col_i) { + columns_to_partition.push_back(input_table_cudf_view.column(this->column_indices_partitioned[col_i])); + } + + cudf::table_view partitioned_table_view(columns_to_partition); + + std::unique_ptr windowed_col; + if (window_expression_contains_partition_by(this->expression)) { + if (is_first_value_window(this->type_aggs_as_str[pos]) || is_last_value_window(this->type_aggs_as_str[pos])) { + + if (is_last_value_window(this->type_aggs_as_str[pos])) { + // We want also all the ordered columns + for (std::size_t col_i = 0; col_i < this->column_indices_ordered.size(); ++col_i) { + columns_to_partition.push_back(input_table_cudf_view.column(this->column_indices_ordered[col_i])); + } + + partitioned_table_view = {{cudf::table_view(columns_to_partition)}}; + } + + // first: we want to get all the first (or last) values (due to the columns to partition) + std::vector requests; + requests.emplace_back(cudf::groupby::aggregation_request()); + requests[0].values = col_view_to_agg; + requests[0].aggregations.push_back(std::move(window_aggregation)); + + cudf::groupby::groupby gb_obj(cudf::table_view({partitioned_table_view}), cudf::null_policy::INCLUDE, cudf::sorted::YES, {}, {}); + std::pair, std::vector> result = gb_obj.aggregate(requests); + + windowed_col = std::move(result.second[0].results[0]); + + // if exists duplicated values (in partitioned_table_view) we want to fill `windowed_col` with repeated values + // So let's do a join + if (windowed_col->size() < col_view_to_agg.size()) { + std::vector> keys_grouped = result.first->release(); + keys_grouped.push_back(std::move(windowed_col)); + + std::unique_ptr left_table = std::make_unique(std::move(keys_grouped)); + + // Let's get all the necessary params for the join + // we just want the key columns, not the values column (which is the last column) + std::vector left_column_indices(left_table->num_columns() - 1); + std::iota(left_column_indices.begin(), left_column_indices.end(), 0); + + std::vector right_column_indices(partitioned_table_view.num_columns()); + std::iota(right_column_indices.begin(), right_column_indices.end(), 0); + + std::unique_ptr join_table = cudf::inner_join( + left_table->view(), + partitioned_table_view, + left_column_indices, + right_column_indices + ); + + // Because the values column is unordered, we want to sort it + std::vector null_orders(join_table->num_columns(), cudf::null_order::AFTER); + + // partition by is always in ASCENDING order + std::vector sortOrderTypes(join_table->num_columns(), cudf::order::ASCENDING); + std::unique_ptr sorted_table = cudf::sort(join_table->view(), sortOrderTypes, null_orders); + + size_t position_of_values_column = left_table->num_columns() - 1; + windowed_col = std::move(sorted_table->release()[position_of_values_column]); + } + } + else if (window_expression_contains_order_by(this->expression)) { + if (window_expression_contains_bounds(this->expression)) { + // TODO: for now just ROWS bounds works (not RANGE) + windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, this->preceding_value + 1, this->following_value, 1, window_aggregation); + } else { + if (this->type_aggs_as_str[pos] == "LEAD") { + windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, 0, col_view_to_agg.size(), 1, window_aggregation); + } else { + windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, col_view_to_agg.size(), 0, 1, window_aggregation); + } + } + } else { + windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, col_view_to_agg.size(), col_view_to_agg.size(), 1, window_aggregation); + } + } else { + if (window_expression_contains_bounds(this->expression)) { + // TODO: for now just ROWS bounds works (not RANGE) + windowed_col = cudf::rolling_window(col_view_to_agg, this->preceding_value + 1, this->following_value, 1, window_aggregation); + } else { + throw std::runtime_error("Window functions without partitions and without bounded windows are currently not supported"); + } + } + + return std::move(windowed_col); +} + +ral::execution::task_result ComputeWindowKernelUnbounded::do_process(std::vector< std::unique_ptr > inputs, + std::shared_ptr output, + cudaStream_t /*stream*/, const std::map& args) { + + + if (inputs.size() == 0) { + return {ral::execution::task_status::SUCCESS, std::string(), std::vector< std::unique_ptr > ()}; + } + + std::unique_ptr & input = inputs[0]; + + try{ + cudf::table_view input_table_cudf_view = input->view(); + + std::vector input_names = input->names(); + + // fill all the Kind aggregations + for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { + AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); + this->aggs_wind_func.push_back(aggr_kind_i); + } + + std::vector< std::unique_ptr > new_wf_cols; + int agg_param_count = 0; + for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { + cudf::column_view col_view_to_agg = input_table_cudf_view.column(column_indices_to_agg[col_i]); + + // calling main window function + std::unique_ptr windowed_col = compute_column_from_window_function(input_table_cudf_view, col_view_to_agg, col_i, agg_param_count); + new_wf_cols.push_back(std::move(windowed_col)); + } + + std::unique_ptr cudf_table_input = input->releaseCudfTable(); + std::vector< std::unique_ptr > input_cudf_columns = cudf_table_input->release(); + + size_t total_output_columns = input_cudf_columns.size() + new_wf_cols.size(); + size_t num_input_cols = input_cudf_columns.size(); + std::vector output_names; + std::vector< std::unique_ptr > output_columns; + + for (size_t col_i = 0; col_i < total_output_columns; ++col_i) { + // appending wf columns + if (col_i >= num_input_cols) { + output_columns.push_back(std::move(new_wf_cols[col_i - num_input_cols])); + output_names.push_back(""); + } else { + output_columns.push_back(std::move(input_cudf_columns[col_i])); + output_names.push_back(input_names[col_i]); + } + } + + std::unique_ptr cudf_table_window = std::make_unique(std::move(output_columns)); + if (this->remove_overlap){ + bool remove_preceding = args.at(TASK_ARG_REMOVE_PRECEDING_OVERLAP) == TRUE; + bool remove_following = args.at(TASK_ARG_REMOVE_FOLLOWING_OVERLAP) == TRUE; + + if (remove_preceding || remove_following) { + std::vector split_indexes; + if (remove_preceding) { + split_indexes.push_back(this->preceding_value); + } + if (remove_following) { + split_indexes.push_back(cudf_table_window->num_rows() - this->following_value); + } + + auto split_window_view = cudf::split(cudf_table_window->view(), split_indexes); + std::unique_ptr temp_table_window; + if (remove_preceding){ + temp_table_window = std::make_unique(split_window_view[1]); + } else { + temp_table_window = std::make_unique(split_window_view[0]); + } + cudf_table_window = std::move(temp_table_window); + } + } + + std::unique_ptr windowed_table = std::make_unique(std::move(cudf_table_window), output_names); + + // WSM TODO get partial_aggregation from windowed_table + + output->addToCache(std::move(windowed_table)); + }catch(const rmm::bad_alloc& e){ + return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; + }catch(const std::exception& e){ + return {ral::execution::task_status::FAIL, std::string(e.what()), std::vector< std::unique_ptr > ()}; + } + + return {ral::execution::task_status::SUCCESS, std::string(), std::vector< std::unique_ptr > ()}; +} + +kstatus ComputeWindowKernelUnbounded::run() { + CodeTimer timer; + + int self_node_idx = context->getNodeIndex(ral::communication::CommunicationData::getInstance().getSelfNode()); + int num_nodes = context->getTotalNodes(); + + bool is_first_batch = true; + bool is_last_batch = false; + bool is_first_node = self_node_idx == 0; + bool is_last_node = self_node_idx == num_nodes - 1; + + std::unique_ptr cache_data = this->input_cache()->pullCacheData(); + + if (this->remove_overlap){ + while (cache_data != nullptr ){ + std::vector > inputs; + inputs.push_back(std::move(cache_data)); + + cache_data = this->input_cache()->pullCacheData(); + if (cache_data == nullptr){ + is_last_batch = true; + } + + std::map task_args; + task_args[TASK_ARG_REMOVE_PRECEDING_OVERLAP] = !(is_first_batch && is_first_node) ? TRUE : FALSE; + task_args[TASK_ARG_REMOVE_FOLLOWING_OVERLAP] = !(is_last_batch && is_last_node) ? TRUE : FALSE; + + ral::execution::executor::get_instance()->add_task( + std::move(inputs), + this->output_cache(), + this, + task_args); + + is_first_batch = false; + } + } else { + while (cache_data != nullptr ){ + std::vector > inputs; + inputs.push_back(std::move(cache_data)); + + ral::execution::executor::get_instance()->add_task( + std::move(inputs), + this->output_cache(), + this); + + cache_data = this->input_cache()->pullCacheData(); + } + } + + + + std::unique_lock lock(kernel_mutex); + kernel_cv.wait(lock,[this]{ + return this->tasks.empty(); + }); + + // WSM TODO here if is master node, wait until we get all partial_aggregations + // Then combine partial_aggregations + // Then send out cummulative_aggregations to nodes + + if (logger != nullptr) { + logger->debug("{query_id}|{step}|{substep}|{info}|{duration}|kernel_id|{kernel_id}||", + "query_id"_a=context->getContextToken(), + "step"_a=context->getQueryStep(), + "substep"_a=context->getQuerySubstep(), + "info"_a="ComputeWindow Kernel Completed", + "duration"_a=timer.elapsed_time(), + "kernel_id"_a=this->get_id()); + } + return kstatus::proceed; +} + +// END ComputeWindowKernelUnbounded + + } // namespace batch } // namespace ral diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index 29a7c065d..2976d5792 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -190,6 +190,46 @@ class OverlapAccumulatorKernel : public distributing_kernel { }; + +/** + * @brief This kernel computes the main Window Function (ROW_NUMBER, LAG, LEAD, MIN, ...) + * to each batch already pattitioned and sorted + * New columns will be added to each batch + */ + +class ComputeWindowKernelUnbounded : public kernel { +public: + ComputeWindowKernelUnbounded(std::size_t kernel_id, const std::string & queryString, + std::shared_ptr context, + std::shared_ptr query_graph); + + std::unique_ptr compute_column_from_window_function( + cudf::table_view input_cudf_view, + cudf::column_view input_col_view, + std::size_t pos, int & agg_param_count); + + std::string kernel_name() { return "ComputeWindowUnbounded";} + + ral::execution::task_result do_process(std::vector< std::unique_ptr > inputs, + std::shared_ptr output, + cudaStream_t stream, const std::map& args) override; + + kstatus run() override; + +private: + std::vector column_indices_partitioned; // column indices to be partitioned: [1] + std::vector column_indices_ordered; // column indices to be ordered: [3] + std::vector column_indices_to_agg; // column indices to be agg: [0, 0] + std::vector agg_param_values; // due to LAG or LEAD: [5] + int preceding_value; // X PRECEDING + int following_value; // Y FOLLOWING + std::string frame_type; // ROWS or RANGE + std::vector type_aggs_as_str; // ["MIN", "LAG"] + std::vector aggs_wind_func; // [AggregateKind::MIN, AggregateKind::LAG] + bool remove_overlap; // If we need to remove the overlaps after computing the windows +}; + + } // namespace batch } // namespace ral From ef264ae351b50a213acb1e8b20a21ab6658ad765 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Tue, 13 Apr 2021 17:16:29 -0500 Subject: [PATCH 02/18] WIP on getting partial aggregations. Also refactored a bit --- .../BatchWindowFunctionProcessing.cpp | 61 +- .../logic_controllers/LogicalProject.cpp | 20 +- .../EndToEndTests/windowFunctionTest.py | 1930 +++++++++-------- 3 files changed, 1037 insertions(+), 974 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index b259104c5..d25db4a88 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -42,6 +42,12 @@ ComputeWindowKernel::ComputeWindowKernel(std::size_t kernel_id, const std::strin std::tie(this->column_indices_partitioned, std::ignore) = ral::operators::get_vars_to_partition(this->expression); std::tie(this->column_indices_ordered, std::ignore) = ral::operators::get_vars_to_orders(this->expression); + // fill all the Kind aggregations + for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { + AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); + this->aggs_wind_func.push_back(aggr_kind_i); + } + // if the window function has no partitioning but does have order by and a bounded window, then we need to remove the overlaps that are present in the data if (column_indices_partitioned.size() == 0 && column_indices_ordered.size() > 0 && this->preceding_value > 0 && this->following_value > 0){ this->remove_overlap = true; @@ -176,12 +182,6 @@ ral::execution::task_result ComputeWindowKernel::do_process(std::vector< std::un std::vector input_names = input->names(); - // fill all the Kind aggregations - for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { - AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); - this->aggs_wind_func.push_back(aggr_kind_i); - } - std::vector< std::unique_ptr > new_wf_cols; int agg_param_count = 0; for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { @@ -1016,6 +1016,12 @@ ComputeWindowKernelUnbounded::ComputeWindowKernelUnbounded(std::size_t kernel_id std::tie(this->column_indices_partitioned, std::ignore) = ral::operators::get_vars_to_partition(this->expression); std::tie(this->column_indices_ordered, std::ignore) = ral::operators::get_vars_to_orders(this->expression); + // fill all the Kind aggregations + for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { + AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); + this->aggs_wind_func.push_back(aggr_kind_i); + } + // if the window function has no partitioning but does have order by and a bounded window, then we need to remove the overlaps that are present in the data if (column_indices_partitioned.size() == 0 && column_indices_ordered.size() > 0 && this->preceding_value > 0 && this->following_value > 0){ this->remove_overlap = true; @@ -1153,12 +1159,6 @@ ral::execution::task_result ComputeWindowKernelUnbounded::do_process(std::vector std::vector input_names = input->names(); - // fill all the Kind aggregations - for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { - AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); - this->aggs_wind_func.push_back(aggr_kind_i); - } - std::vector< std::unique_ptr > new_wf_cols; int agg_param_count = 0; for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { @@ -1176,12 +1176,14 @@ ral::execution::task_result ComputeWindowKernelUnbounded::do_process(std::vector size_t num_input_cols = input_cudf_columns.size(); std::vector output_names; std::vector< std::unique_ptr > output_columns; + std::vector window_func_col_indices; for (size_t col_i = 0; col_i < total_output_columns; ++col_i) { // appending wf columns if (col_i >= num_input_cols) { output_columns.push_back(std::move(new_wf_cols[col_i - num_input_cols])); output_names.push_back(""); + window_func_col_indices.push_back(col_i); } else { output_columns.push_back(std::move(input_cudf_columns[col_i])); output_names.push_back(input_names[col_i]); @@ -1215,7 +1217,40 @@ ral::execution::task_result ComputeWindowKernelUnbounded::do_process(std::vector std::unique_ptr windowed_table = std::make_unique(std::move(cudf_table_window), output_names); - // WSM TODO get partial_aggregation from windowed_table + // // get partial_aggregations from windowed_table + // if (this->aggs_wind_func == AggregateKind::RANK){ + // // TODO throw error. not yet implemented + // // partial_aggregation = last rank value of batch + number of elements with that rank value - 1 + // } else if (this->aggs_wind_func == AggregateKind::DENSE_RANK){ + // // TODO throw error. not yet implemented + // // partial_aggregation = last dense rank value of batch + // } else { + auto windowed_table_cudf_view = windowed_table->view(); + std::vector window_agg_columns; + std::vector window_agg_column_names; + for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { + window_agg_columns.push_back(windowed_table_cudf_view.column(window_func_col_indices[col_i])); + window_agg_column_names.push_back(windowed_table->names()[window_func_col_indices[col_i]]); + } + cudf::table_view window_agg_table_view(window_agg_columns); + cudf::table_view partial_aggregations_table_view; + if (window_agg_table_view.num_rows() > 2){ + if (preceding is unbounded) { + // want to get last value + std::vector split_window_view = cudf::split(window_agg_table_view, {window_agg_table_view.num_rows() - 2}); + partial_aggregations_table_view = split_window_view[1]; + } else if (following is unbounded) { + // want to get first value + std::vector split_window_view = cudf::split(window_agg_table_view, {1}); + partial_aggregations_table_view = split_window_view[0]; + } else { + // error one of the two should be unbounded otherwise we should not be in this kernel + } + } else { + partial_aggregations_table_view = window_agg_table_view; + } + + // } output->addToCache(std::move(windowed_table)); }catch(const rmm::bad_alloc& e){ diff --git a/engine/src/execution_graph/logic_controllers/LogicalProject.cpp b/engine/src/execution_graph/logic_controllers/LogicalProject.cpp index 1d3cab704..31c91291a 100644 --- a/engine/src/execution_graph/logic_controllers/LogicalProject.cpp +++ b/engine/src/execution_graph/logic_controllers/LogicalProject.cpp @@ -787,11 +787,7 @@ std::vector> evaluate_expressions( const std::vector & expressions) { using interops::column_index_type; - // Let's clean all the expressions that contains Window functions (if exists) - // as they should be updated with new indices - std::vector new_expressions = clean_window_function_expressions(expressions, table.num_columns()); - - std::vector> out_columns(new_expressions.size()); + std::vector> out_columns(expressions.size()); std::vector column_used(table.num_columns(), false); std::vector> out_idx_computed_idx_pair; @@ -800,8 +796,8 @@ std::vector> evaluate_expressions( std::vector interpreter_out_column_views; function_evaluator_transformer evaluator{table}; - for(size_t i = 0; i < new_expressions.size(); i++){ - std::string expression = replace_calcite_regex(new_expressions[i]); + for(size_t i = 0; i < expressions.size(); i++){ + std::string expression = replace_calcite_regex(expressions[i]); expression = expand_if_logical_op(expression); parser::parse_tree tree; @@ -904,9 +900,9 @@ std::vector> evaluate_expressions( out_columns.clear(); computed_columns.clear(); - size_t const half_size = new_expressions.size() / 2; - std::vector split_lo(new_expressions.begin(), new_expressions.begin() + half_size); - std::vector split_hi(new_expressions.begin() + half_size, new_expressions.end()); + size_t const half_size = expressions.size() / 2; + std::vector split_lo(expressions.begin(), expressions.begin() + half_size); + std::vector split_hi(expressions.begin() + half_size, expressions.end()); auto out_cols_lo = evaluate_expressions(table, split_lo); auto out_cols_hi = evaluate_expressions(table, split_hi); @@ -955,6 +951,10 @@ std::unique_ptr process_project( out_column_names[i] = name; } + // Let's clean all the expressions that contains Window functions (if exists) + // as they should be updated with new indices + expressions = clean_window_function_expressions(expressions, blazing_table_in->num_columns()); + return std::make_unique(evaluate_expressions(blazing_table_in->view(), expressions), out_column_names); } diff --git a/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py b/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py index e3f85724b..25f338ac2 100644 --- a/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py +++ b/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py @@ -16,10 +16,10 @@ def main(dask_client, drill, spark, dir_data_file, bc, nRals): def executionTest(): tables = ["orders", "nation", "lineitem", "customer", "region"] data_types = [ - DataType.DASK_CUDF, - DataType.CUDF, - DataType.CSV, - DataType.ORC, + # DataType.DASK_CUDF, + # DataType.CUDF, + # DataType.CSV, + # DataType.ORC, DataType.PARQUET, ] # TODO json @@ -232,72 +232,72 @@ def executionTest(): # ----------------- PARTITION BY ---------------------- - queryId = "TEST_11" - query = """select min(n_nationkey) over - ( - partition by n_regionkey - ) min_keys, - n_nationkey, n_name, n_regionkey - from nation order by n_name""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_11" + # query = """select min(n_nationkey) over + # ( + # partition by n_regionkey + # ) min_keys, + # n_nationkey, n_name, n_regionkey + # from nation order by n_name""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_12" - query = """select max(o_orderkey) over - ( - partition by o_custkey - ) max_keys, - o_custkey, o_orderstatus, o_totalprice - from orders - where o_orderstatus = 'O' - and o_totalprice > 98000.5 - order by o_orderkey, o_totalprice - limit 1200""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_12" + # query = """select max(o_orderkey) over + # ( + # partition by o_custkey + # ) max_keys, + # o_custkey, o_orderstatus, o_totalprice + # from orders + # where o_orderstatus = 'O' + # and o_totalprice > 98000.5 + # order by o_orderkey, o_totalprice + # limit 1200""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_13" - query = """select count(l_orderkey) over - ( - partition by l_partkey - ) count_keys, - l_linenumber, l_extendedprice, l_tax - from lineitem - where l_partkey < 250 - and l_linenumber > 4 - order by l_orderkey, l_partkey""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_13" + # query = """select count(l_orderkey) over + # ( + # partition by l_partkey + # ) count_keys, + # l_linenumber, l_extendedprice, l_tax + # from lineitem + # where l_partkey < 250 + # and l_linenumber > 4 + # order by l_orderkey, l_partkey""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) queryId = "TEST_14" query = """select sum(o_custkey) over @@ -310,30 +310,7 @@ def executionTest(): where o_orderstatus <> 'O' and o_totalprice > 35000 and o_orderpriority = '3-MEDIUM' - order by o_orderpriority""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) - - queryId = "TEST_15" - query = """select avg(cast(n.n_nationkey as double)) over - ( - partition by n.n_regionkey - ) avg_keys, - n.n_nationkey, n.n_name, n.n_regionkey, l.l_comment - from nation as n - inner join lineitem as l - on n.n_nationkey = l.l_orderkey - order by n.n_nationkey, avg_keys""" + """ runTest.run_query( bc, drill, @@ -346,20 +323,21 @@ def executionTest(): use_percentage, fileSchemaType, ) + print(bc.explain(query, detail=True)) - queryId = "TEST_16" - query = """select sum(o_custkey) over - ( - partition by o_orderpriority, o_orderstatus - ) sum_keys, - o_clerk, + + queryId = "TEST_14C" + query = """select o_clerk, cast(o_shippriority as double) as o_ship_double, - o_orderpriority + sum(o_custkey) over + ( + partition by o_orderpriority + ) sum_keys from orders - where o_orderstatus = 'O' - and o_totalprice < 4550 - and o_orderpriority = '1-URGENT' - order by o_orderpriority""" + where o_orderstatus <> 'O' + and o_totalprice > 35000 + and o_orderpriority = '3-MEDIUM' + """ runTest.run_query( bc, drill, @@ -372,888 +350,938 @@ def executionTest(): use_percentage, fileSchemaType, ) + print(bc.explain(query, detail=True)) - queryId = "TEST_17" - query = """select min(c_custkey) over - ( - partition by c_mktsegment, c_name, c_custkey - ) min_keys, - c_custkey, c_mktsegment - from customer - where c_acctbal > 4525.0 - and c_mktsegment not in ('AUTOMOBILE', 'HOUSEHOLD') - order by c_custkey, c_nationkey - limit 250""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_15" + # query = """select avg(cast(n.n_nationkey as double)) over + # ( + # partition by n.n_regionkey + # ) avg_keys, + # n.n_nationkey, n.n_name, n.n_regionkey, l.l_comment + # from nation as n + # inner join lineitem as l + # on n.n_nationkey = l.l_orderkey + # order by n.n_nationkey, avg_keys""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_18" - query = """select avg(cast(n.n_nationkey as double)) over - ( - partition by n.n_regionkey, n.n_name, n.n_nationkey - ) avg_keys, - n.n_nationkey, n.n_name, n.n_regionkey, l.l_comment - from nation as n - inner join lineitem as l - on n.n_nationkey = l.l_orderkey - order by n.n_nationkey, avg_keys""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_16" + # query = """select sum(o_custkey) over + # ( + # partition by o_orderpriority, o_orderstatus + # ) sum_keys, + # o_clerk, + # cast(o_shippriority as double) as o_ship_double, + # o_orderpriority + # from orders + # where o_orderstatus = 'O' + # and o_totalprice < 4550 + # and o_orderpriority = '1-URGENT' + # order by o_orderpriority""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_19" - query = """with first_window_order as ( - select max(o_totalprice) over - ( - partition by o_orderpriority - order by o_totalprice, o_custkey - ) o_max_prices, - min(o_totalprice) over - ( - partition by o_orderpriority - order by o_totalprice, o_custkey - ) o_min_prices, - o_orderkey, o_orderpriority, o_custkey, - o_totalprice, o_clerk - from orders - ), order_operated as ( - select * from first_window_order - where o_max_prices < 19750.0 - and o_clerk <> 'Clerk#000000880' - and o_orderpriority in ('2-HIGH', '5-LOW') - order by o_orderkey, o_custkey, o_totalprice - limit 1250 - ) - select sum(o_max_prices) over - ( - partition by o_orderpriority - order by o_totalprice, o_custkey - ) sum_max_prices, - o_orderkey, o_min_prices, o_orderpriority from order_operated - order by o_orderkey, o_min_prices, o_totalprice - limit 450""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_17" + # query = """select min(c_custkey) over + # ( + # partition by c_mktsegment, c_name, c_custkey + # ) min_keys, + # c_custkey, c_mktsegment + # from customer + # where c_acctbal > 4525.0 + # and c_mktsegment not in ('AUTOMOBILE', 'HOUSEHOLD') + # order by c_custkey, c_nationkey + # limit 250""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_20" - query = """with reduced_order as ( - select o_orderkey, o_custkey, o_orderpriority, - o_orderstatus, o_totalprice, o_clerk - from orders - where o_totalprice < 1750.0 - and o_clerk <> 'Clerk#000000880' - order by o_orderkey, o_custkey, o_totalprice - limit 3500 - ), window_orders as ( - select min(o_totalprice) over - ( - partition by o_orderpriority - order by o_totalprice - ) o_min_prices, - o_orderkey, o_orderpriority, o_orderstatus - from reduced_order - ) - select o_orderkey, o_min_prices, o_orderpriority - from window_orders - where o_orderstatus in ('O', 'F') - and o_orderpriority = '2-HIGH' - order by o_orderkey, o_min_prices""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_18" + # query = """select avg(cast(n.n_nationkey as double)) over + # ( + # partition by n.n_regionkey, n.n_name, n.n_nationkey + # ) avg_keys, + # n.n_nationkey, n.n_name, n.n_regionkey, l.l_comment + # from nation as n + # inner join lineitem as l + # on n.n_nationkey = l.l_orderkey + # order by n.n_nationkey, avg_keys""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - # ------------ PARTITION BY + ORDER BY ---------------- + # queryId = "TEST_19" + # query = """with first_window_order as ( + # select max(o_totalprice) over + # ( + # partition by o_orderpriority + # order by o_totalprice, o_custkey + # ) o_max_prices, + # min(o_totalprice) over + # ( + # partition by o_orderpriority + # order by o_totalprice, o_custkey + # ) o_min_prices, + # o_orderkey, o_orderpriority, o_custkey, + # o_totalprice, o_clerk + # from orders + # ), order_operated as ( + # select * from first_window_order + # where o_max_prices < 19750.0 + # and o_clerk <> 'Clerk#000000880' + # and o_orderpriority in ('2-HIGH', '5-LOW') + # order by o_orderkey, o_custkey, o_totalprice + # limit 1250 + # ) + # select sum(o_max_prices) over + # ( + # partition by o_orderpriority + # order by o_totalprice, o_custkey + # ) sum_max_prices, + # o_orderkey, o_min_prices, o_orderpriority from order_operated + # order by o_orderkey, o_min_prices, o_totalprice + # limit 450""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_21" - query = """select min(c_custkey) over - ( - partition by c_nationkey - order by c_name - ) min_keys, - c_custkey, c_mktsegment - from customer - where c_acctbal > 2525.0 - and c_mktsegment not in ('AUTOMOBILE', 'HOUSEHOLD') - order by c_custkey, c_nationkey""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_20" + # query = """with reduced_order as ( + # select o_orderkey, o_custkey, o_orderpriority, + # o_orderstatus, o_totalprice, o_clerk + # from orders + # where o_totalprice < 1750.0 + # and o_clerk <> 'Clerk#000000880' + # order by o_orderkey, o_custkey, o_totalprice + # limit 3500 + # ), window_orders as ( + # select min(o_totalprice) over + # ( + # partition by o_orderpriority + # order by o_totalprice + # ) o_min_prices, + # o_orderkey, o_orderpriority, o_orderstatus + # from reduced_order + # ) + # select o_orderkey, o_min_prices, o_orderpriority + # from window_orders + # where o_orderstatus in ('O', 'F') + # and o_orderpriority = '2-HIGH' + # order by o_orderkey, o_min_prices""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_22" - query = """select max(l_partkey) over - ( - partition by l_linestatus - order by l_quantity desc, l_orderkey - ) max_keys, - l_linestatus, l_extendedprice - from lineitem - where l_shipmode not in ('MAIL', 'SHIP', 'AIR') - and l_linestatus = 'F' - order by l_orderkey, max_keys - limit 50""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # # ------------ PARTITION BY + ORDER BY ---------------- - queryId = "TEST_23" - query = """select count(o_orderkey) over - ( - partition by o_orderstatus, o_orderpriority - order by o_orderkey, o_clerk - ) count_keys, - o_totalprice - from orders - where o_totalprice < 1352.0 - order by o_custkey, o_orderpriority, o_orderkey - limit 50""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_21" + # query = """select min(c_custkey) over + # ( + # partition by c_nationkey + # order by c_name + # ) min_keys, + # c_custkey, c_mktsegment + # from customer + # where c_acctbal > 2525.0 + # and c_mktsegment not in ('AUTOMOBILE', 'HOUSEHOLD') + # order by c_custkey, c_nationkey""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_24" - query = """select sum(n.n_nationkey) over - ( - partition by n.n_regionkey - order by n.n_nationkey desc, n.n_name desc - ) sum_keys, - n.n_nationkey, c.c_address, c.c_custkey - from nation n - inner join customer c on n.n_nationkey = c.c_custkey - where c.c_mktsegment <> 'household' - and n.n_nationkey in (1, 2, 3, 8, 9, 12)""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) - - queryId = "TEST_25" - query = """with new_nation as ( - select n.n_nationkey as n_natio1, - n.n_name as n_nam1, - n.n_regionkey as n_region1 - from nation as n - inner join region as r - on n.n_nationkey = r.r_regionkey - ) - select avg(cast(nn.n_natio1 as double)) over - ( - partition by nn.n_region1 - order by nn.n_nam1 - ) avg_keys, - nn.n_natio1, nn.n_nam1, nn.n_region1 - from new_nation nn - order by nn.n_natio1, avg_keys""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_22" + # query = """select max(l_partkey) over + # ( + # partition by l_linestatus + # order by l_quantity desc, l_orderkey + # ) max_keys, + # l_linestatus, l_extendedprice + # from lineitem + # where l_shipmode not in ('MAIL', 'SHIP', 'AIR') + # and l_linestatus = 'F' + # order by l_orderkey, max_keys + # limit 50""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_26" - query = """select row_number() over - ( - partition by c_nationkey - order by c_custkey desc, c_acctbal - ) row_num, - c_phone, UPPER(SUBSTRING(c_name, 1, 8)) - from customer - where c_acctbal < 95.0 - order by row_num, c_acctbal""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_23" + # query = """select count(o_orderkey) over + # ( + # partition by o_orderstatus, o_orderpriority + # order by o_orderkey, o_clerk + # ) count_keys, + # o_totalprice + # from orders + # where o_totalprice < 1352.0 + # order by o_custkey, o_orderpriority, o_orderkey + # limit 50""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_27" - query = """select row_number() over - ( - partition by c_nationkey, c_mktsegment - order by c_custkey desc, c_name, c_phone - ) row_num, - c_phone, UPPER(SUBSTRING(c_name, 1, 8)) - from customer - where c_acctbal < 155.0 - order by c_custkey, row_num desc""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_24" + # query = """select sum(n.n_nationkey) over + # ( + # partition by n.n_regionkey + # order by n.n_nationkey desc, n.n_name desc + # ) sum_keys, + # n.n_nationkey, c.c_address, c.c_custkey + # from nation n + # inner join customer c on n.n_nationkey = c.c_custkey + # where c.c_mktsegment <> 'household' + # and n.n_nationkey in (1, 2, 3, 8, 9, 12)""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) + + # queryId = "TEST_25" + # query = """with new_nation as ( + # select n.n_nationkey as n_natio1, + # n.n_name as n_nam1, + # n.n_regionkey as n_region1 + # from nation as n + # inner join region as r + # on n.n_nationkey = r.r_regionkey + # ) + # select avg(cast(nn.n_natio1 as double)) over + # ( + # partition by nn.n_region1 + # order by nn.n_nam1 + # ) avg_keys, + # nn.n_natio1, nn.n_nam1, nn.n_region1 + # from new_nation nn + # order by nn.n_natio1, avg_keys""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_28" - query = """select lag(l_partkey, 2) over - ( - partition by l_linestatus - order by l_orderkey, l_quantity desc - ) lag_keys, - l_linestatus, l_partkey, l_extendedprice - from lineitem - where l_partkey < 750 - and l_linenumber >= 6 - order by l_extendedprice, l_partkey, lag_keys - limit 30""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_26" + # query = """select row_number() over + # ( + # partition by c_nationkey + # order by c_custkey desc, c_acctbal + # ) row_num, + # c_phone, UPPER(SUBSTRING(c_name, 1, 8)) + # from customer + # where c_acctbal < 95.0 + # order by row_num, c_acctbal""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_29" - query = """select lead(l_partkey, 3) over - ( - partition by l_linestatus - order by l_extendedprice - ) lead_keys, - l_linestatus, l_partkey, l_extendedprice - from lineitem - where l_partkey < 950 - and l_linenumber >= 7 - order by l_extendedprice, l_partkey - limit 40""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_27" + # query = """select row_number() over + # ( + # partition by c_nationkey, c_mktsegment + # order by c_custkey desc, c_name, c_phone + # ) row_num, + # c_phone, UPPER(SUBSTRING(c_name, 1, 8)) + # from customer + # where c_acctbal < 155.0 + # order by c_custkey, row_num desc""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_30" - query = """select (o_orderkey + o_custkey) as key_priority, - max(o_totalprice) over - ( - partition by o_orderpriority - order by o_totalprice, o_custkey - ) o_max_prices, - o_custkey + o_totalprice, - min(o_totalprice) over - ( - partition by o_orderpriority - order by o_totalprice, o_custkey - ) o_min_prices, - o_custkey - o_totalprice + 5 - from orders - where o_orderstatus not in ('O', 'F') - and o_totalprice < 85000 - and o_orderpriority <> '2-HIGH' - order by key_priority, o_max_prices""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_28" + # query = """select lag(l_partkey, 2) over + # ( + # partition by l_linestatus + # order by l_orderkey, l_quantity desc + # ) lag_keys, + # l_linestatus, l_partkey, l_extendedprice + # from lineitem + # where l_partkey < 750 + # and l_linenumber >= 6 + # order by l_extendedprice, l_partkey, lag_keys + # limit 30""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - # ---------- multiple WF with the same OVER clause ------------ + # queryId = "TEST_29" + # query = """select lead(l_partkey, 3) over + # ( + # partition by l_linestatus + # order by l_extendedprice + # ) lead_keys, + # l_linestatus, l_partkey, l_extendedprice + # from lineitem + # where l_partkey < 950 + # and l_linenumber >= 7 + # order by l_extendedprice, l_partkey + # limit 40""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_31" - query = """select min(o_orderkey) over - ( - partition by o_orderstatus - order by o_orderdate - ) min_keys, - max(o_orderkey) over - ( - partition by o_orderstatus - order by o_orderdate - ) max_keys, o_orderkey, o_orderpriority - from orders - where o_orderpriority <> '2-HIGH' - and o_clerk = 'Clerk#000000880' - order by o_orderkey""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_30" + # query = """select (o_orderkey + o_custkey) as key_priority, + # max(o_totalprice) over + # ( + # partition by o_orderpriority + # order by o_totalprice, o_custkey + # ) o_max_prices, + # o_custkey + o_totalprice, + # min(o_totalprice) over + # ( + # partition by o_orderpriority + # order by o_totalprice, o_custkey + # ) o_min_prices, + # o_custkey - o_totalprice + 5 + # from orders + # where o_orderstatus not in ('O', 'F') + # and o_totalprice < 85000 + # and o_orderpriority <> '2-HIGH' + # order by key_priority, o_max_prices""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_32" - query = """select min(n_nationkey) over - ( - partition by n_regionkey - order by n_name - ) min_keys, - max(n_nationkey) over - ( - partition by n_regionkey - order by n_name - ) max_keys, - count(n_nationkey) over - ( - partition by n_regionkey - order by n_name - ) count_keys, n_nationkey, n_name, n_regionkey - from nation order by n_nationkey""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # # ---------- multiple WF with the same OVER clause ------------ - queryId = "TEST_33" - query = """select min(o_orderkey) over - ( - partition by o_orderstatus, o_clerk - order by o_orderdate - ) min_keys, - max(o_orderkey) over - ( - partition by o_orderstatus, o_clerk - order by o_orderdate - ) max_keys, o_orderkey, o_orderpriority - from orders - where o_orderpriority <> '2-HIGH' - and o_clerk = 'Clerk#000000880' - order by o_orderkey""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_31" + # query = """select min(o_orderkey) over + # ( + # partition by o_orderstatus + # order by o_orderdate + # ) min_keys, + # max(o_orderkey) over + # ( + # partition by o_orderstatus + # order by o_orderdate + # ) max_keys, o_orderkey, o_orderpriority + # from orders + # where o_orderpriority <> '2-HIGH' + # and o_clerk = 'Clerk#000000880' + # order by o_orderkey""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - # NOTE: order by in the over clause is mandatory for spark - queryId = "TEST_34" - query = """select min(n_nationkey) over - ( - partition by n_regionkey order by n_name - ) min_keys, - lag(n_nationkey, 2) over - ( - partition by n_regionkey order by n_name - ) lag_col, - max(n_nationkey) over - ( - partition by n_regionkey order by n_name - ) max_keys, - n_nationkey, n_name, n_regionkey - from nation order by n_nationkey""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_32" + # query = """select min(n_nationkey) over + # ( + # partition by n_regionkey + # order by n_name + # ) min_keys, + # max(n_nationkey) over + # ( + # partition by n_regionkey + # order by n_name + # ) max_keys, + # count(n_nationkey) over + # ( + # partition by n_regionkey + # order by n_name + # ) count_keys, n_nationkey, n_name, n_regionkey + # from nation order by n_nationkey""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_35" - query = """select sum(o_custkey) over - ( - partition by o_orderstatus, o_orderpriority - order by o_totalprice, o_custkey - ) sum_keys, - lag(o_custkey, 2) over - ( - partition by o_orderstatus, o_orderpriority - order by o_totalprice, o_custkey - ) lag_keys, - cast(o_shippriority as double) as o_ship_double, - o_orderpriority - from orders - where o_orderstatus <> 'O' - and o_totalprice <= 6000 - and o_orderpriority in ('2-HIGH', '1-URGENT') - order by o_orderpriority, o_totalprice""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_33" + # query = """select min(o_orderkey) over + # ( + # partition by o_orderstatus, o_clerk + # order by o_orderdate + # ) min_keys, + # max(o_orderkey) over + # ( + # partition by o_orderstatus, o_clerk + # order by o_orderdate + # ) max_keys, o_orderkey, o_orderpriority + # from orders + # where o_orderpriority <> '2-HIGH' + # and o_clerk = 'Clerk#000000880' + # order by o_orderkey""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_36" - query = """select sum(o_custkey) over - ( - partition by o_orderstatus, o_orderpriority - order by o_totalprice, o_custkey - ) sum_keys, - lead(o_custkey, 3) over - ( - partition by o_orderstatus, o_orderpriority - order by o_totalprice, o_custkey - ) lead_keys, - cast(o_shippriority as double) as o_ship_double, - o_orderpriority - from orders - where o_orderstatus <> 'O' - and o_totalprice <= 6000 - and o_orderpriority in ('2-HIGH', '1-URGENT') - order by o_orderpriority""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # # NOTE: order by in the over clause is mandatory for spark + # queryId = "TEST_34" + # query = """select min(n_nationkey) over + # ( + # partition by n_regionkey order by n_name + # ) min_keys, + # lag(n_nationkey, 2) over + # ( + # partition by n_regionkey order by n_name + # ) lag_col, + # max(n_nationkey) over + # ( + # partition by n_regionkey order by n_name + # ) max_keys, + # n_nationkey, n_name, n_regionkey + # from nation order by n_nationkey""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_37" - query = """select - max(n.n_nationkey) over - ( - partition by l.l_partkey - order by l.l_extendedprice - ) max_keys, - lead(n.n_nationkey, 2) over - ( - partition by l.l_partkey - order by l.l_extendedprice - ) lead_keys, - n.n_nationkey, l.l_extendedprice, l.l_comment - from nation as n - inner join lineitem as l - on n.n_nationkey = l.l_partkey - order by l.l_extendedprice, l_comment - limit 10""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_35" + # query = """select sum(o_custkey) over + # ( + # partition by o_orderstatus, o_orderpriority + # order by o_totalprice, o_custkey + # ) sum_keys, + # lag(o_custkey, 2) over + # ( + # partition by o_orderstatus, o_orderpriority + # order by o_totalprice, o_custkey + # ) lag_keys, + # cast(o_shippriority as double) as o_ship_double, + # o_orderpriority + # from orders + # where o_orderstatus <> 'O' + # and o_totalprice <= 6000 + # and o_orderpriority in ('2-HIGH', '1-URGENT') + # order by o_orderpriority, o_totalprice""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - # using the same column `c_nationkey` to partition - # and first_value() - queryId = "TEST_38" - query = """select first_value(c_nationkey) over - ( - partition by c_nationkey - order by c_custkey desc - ) row_num, - c_phone, UPPER(SUBSTRING(c_name, 1, 7)) - from customer - where c_acctbal < 125.0 - order by c_custkey, row_num""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_36" + # query = """select sum(o_custkey) over + # ( + # partition by o_orderstatus, o_orderpriority + # order by o_totalprice, o_custkey + # ) sum_keys, + # lead(o_custkey, 3) over + # ( + # partition by o_orderstatus, o_orderpriority + # order by o_totalprice, o_custkey + # ) lead_keys, + # cast(o_shippriority as double) as o_ship_double, + # o_orderpriority + # from orders + # where o_orderstatus <> 'O' + # and o_totalprice <= 6000 + # and o_orderpriority in ('2-HIGH', '1-URGENT') + # order by o_orderpriority""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) + + # queryId = "TEST_37" + # query = """select + # max(n.n_nationkey) over + # ( + # partition by l.l_partkey + # order by l.l_extendedprice + # ) max_keys, + # lead(n.n_nationkey, 2) over + # ( + # partition by l.l_partkey + # order by l.l_extendedprice + # ) lead_keys, + # n.n_nationkey, l.l_extendedprice, l.l_comment + # from nation as n + # inner join lineitem as l + # on n.n_nationkey = l.l_partkey + # order by l.l_extendedprice, l_comment + # limit 10""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - # using the same column `c_nationkey` to partition - # and last_value() - queryId = "TEST_39" - query = """select last_value(c_nationkey) over - ( - partition by c_nationkey - order by c_custkey desc - ) row_num, - c_phone, UPPER(SUBSTRING(c_name, 1, 7)) - from customer - where c_acctbal < 125.0 - order by c_custkey, row_num""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # # using the same column `c_nationkey` to partition + # # and first_value() + # queryId = "TEST_38" + # query = """select first_value(c_nationkey) over + # ( + # partition by c_nationkey + # order by c_custkey desc + # ) row_num, + # c_phone, UPPER(SUBSTRING(c_name, 1, 7)) + # from customer + # where c_acctbal < 125.0 + # order by c_custkey, row_num""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_40" - query = """select - max(n.n_nationkey) over - ( - partition by l.l_partkey - order by l.l_extendedprice - ) max_keys, - lead(n.n_nationkey, 2) over - ( - partition by l.l_partkey - order by l.l_extendedprice - ) lead_keys, - first_value(n.n_nationkey) over - ( - partition by l.l_partkey - order by l.l_extendedprice - ) first_keys, - last_value(n.n_nationkey) over - ( - partition by l.l_partkey - order by l.l_extendedprice - ) last_keys, - n.n_nationkey, l.l_extendedprice, l.l_comment - from nation as n - inner join lineitem as l - on n.n_nationkey = l.l_partkey - order by l.l_extendedprice, l_comment - limit 10""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # # using the same column `c_nationkey` to partition + # # and last_value() + # queryId = "TEST_39" + # query = """select last_value(c_nationkey) over + # ( + # partition by c_nationkey + # order by c_custkey desc + # ) row_num, + # c_phone, UPPER(SUBSTRING(c_name, 1, 7)) + # from customer + # where c_acctbal < 125.0 + # order by c_custkey, row_num""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_41" - query = """select (o_orderkey + o_custkey) as key_priority, - o_custkey + o_totalprice, - first_value(o_totalprice) over - ( - partition by o_orderpriority - order by o_totalprice, o_custkey - ) o_first_prices, - o_custkey - o_totalprice + 6 - from orders - where o_orderstatus not in ('O', 'F') - and o_totalprice < 55000 - and o_orderpriority <> '2-HIGH' - order by key_priority, o_first_prices""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_40" + # query = """select + # max(n.n_nationkey) over + # ( + # partition by l.l_partkey + # order by l.l_extendedprice + # ) max_keys, + # lead(n.n_nationkey, 2) over + # ( + # partition by l.l_partkey + # order by l.l_extendedprice + # ) lead_keys, + # first_value(n.n_nationkey) over + # ( + # partition by l.l_partkey + # order by l.l_extendedprice + # ) first_keys, + # last_value(n.n_nationkey) over + # ( + # partition by l.l_partkey + # order by l.l_extendedprice + # ) last_keys, + # n.n_nationkey, l.l_extendedprice, l.l_comment + # from nation as n + # inner join lineitem as l + # on n.n_nationkey = l.l_partkey + # order by l.l_extendedprice, l_comment + # limit 10""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - # ------------ ROWS bounding ---------------- + # queryId = "TEST_41" + # query = """select (o_orderkey + o_custkey) as key_priority, + # o_custkey + o_totalprice, + # first_value(o_totalprice) over + # ( + # partition by o_orderpriority + # order by o_totalprice, o_custkey + # ) o_first_prices, + # o_custkey - o_totalprice + 6 + # from orders + # where o_orderstatus not in ('O', 'F') + # and o_totalprice < 55000 + # and o_orderpriority <> '2-HIGH' + # order by key_priority, o_first_prices""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_50" - query = """select min(n_nationkey) over - ( - partition by n_regionkey - order by n_name - ROWS BETWEEN 1 PRECEDING - AND 1 FOLLOWING - ) min_val, - n_nationkey, n_regionkey, n_name - from nation order by n_nationkey""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # # ------------ ROWS bounding ---------------- - queryId = "TEST_51" - query = """select min(o_orderkey) over - ( - partition by o_orderstatus, o_orderpriority - order by o_totalprice - ROWS BETWEEN 2 PRECEDING - AND 1 FOLLOWING - ) min_keys, - max(o_orderkey) over - ( - partition by o_orderstatus, o_orderpriority - order by o_totalprice - ROWS BETWEEN 2 PRECEDING - AND 1 FOLLOWING - ) max_keys, o_orderkey, o_orderpriority - from orders - where o_orderpriority <> '2-HIGH' - and o_clerk = 'Clerk#000000880' - and o_orderstatus is not null - and o_totalprice is not null - order by o_orderstatus, o_totalprice - limit 50""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_50" + # query = """select min(n_nationkey) over + # ( + # partition by n_regionkey + # order by n_name + # ROWS BETWEEN 1 PRECEDING + # AND 1 FOLLOWING + # ) min_val, + # n_nationkey, n_regionkey, n_name + # from nation order by n_nationkey""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_52" - query = """with new_nation as ( - select n.n_nationkey as n_natio1, - n.n_name as n_nam1, - n.n_regionkey as n_region1 - from nation as n - inner join region as r - on n.n_nationkey = r.r_regionkey - ) - select avg(cast(nn.n_natio1 as double)) over - ( - partition by nn.n_region1 - order by nn.n_nam1 - ROWS BETWEEN 3 PRECEDING - AND 2 FOLLOWING - ) avg_keys, - nn.n_natio1, nn.n_nam1, nn.n_region1 - from new_nation nn - order by nn.n_natio1, avg_keys""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_51" + # query = """select min(o_orderkey) over + # ( + # partition by o_orderstatus, o_orderpriority + # order by o_totalprice + # ROWS BETWEEN 2 PRECEDING + # AND 1 FOLLOWING + # ) min_keys, + # max(o_orderkey) over + # ( + # partition by o_orderstatus, o_orderpriority + # order by o_totalprice + # ROWS BETWEEN 2 PRECEDING + # AND 1 FOLLOWING + # ) max_keys, o_orderkey, o_orderpriority + # from orders + # where o_orderpriority <> '2-HIGH' + # and o_clerk = 'Clerk#000000880' + # and o_orderstatus is not null + # and o_totalprice is not null + # order by o_orderstatus, o_totalprice + # limit 50""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_53" - query = """select max(l_partkey) over - ( - partition by l_suppkey, l_partkey, l_linenumber - order by l_extendedprice desc, l_orderkey, l_quantity - ROWS BETWEEN 6 PRECEDING - AND 2 FOLLOWING - ) max_keys, - l_linestatus, l_extendedprice - from lineitem - where l_shipmode not in ('MAIL', 'SHIP', 'AIR') - and l_linestatus = 'F' - -- as pyspark put the nulls at the begining and blazing to the end - -- we want make sure the orders are the same - and l_extendedprice is not null - order by l_extendedprice, l_orderkey, max_keys - limit 50""" + # queryId = "TEST_52" + # query = """with new_nation as ( + # select n.n_nationkey as n_natio1, + # n.n_name as n_nam1, + # n.n_regionkey as n_region1 + # from nation as n + # inner join region as r + # on n.n_nationkey = r.r_regionkey + # ) + # select avg(cast(nn.n_natio1 as double)) over + # ( + # partition by nn.n_region1 + # order by nn.n_nam1 + # ROWS BETWEEN 3 PRECEDING + # AND 2 FOLLOWING + # ) avg_keys, + # nn.n_natio1, nn.n_nam1, nn.n_region1 + # from new_nation nn + # order by nn.n_natio1, avg_keys""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # queryId = "TEST_53" + # query = """select max(l_partkey) over + # ( + # partition by l_suppkey, l_partkey, l_linenumber + # order by l_extendedprice desc, l_orderkey, l_quantity + # ROWS BETWEEN 6 PRECEDING + # AND 2 FOLLOWING + # ) max_keys, + # l_linestatus, l_extendedprice + # from lineitem + # where l_shipmode not in ('MAIL', 'SHIP', 'AIR') + # and l_linestatus = 'F' + # -- as pyspark put the nulls at the begining and blazing to the end + # -- we want make sure the orders are the same + # and l_extendedprice is not null + # order by l_extendedprice, l_orderkey, max_keys + # limit 50""" - # using diffs columns to partition and first_value() - queryId = "TEST_54" - query = """select first_value(c_custkey) over - ( - partition by c_nationkey - order by c_name desc - ) row_num, - c_phone, UPPER(SUBSTRING(c_name, 1, 7)) - from customer - where c_acctbal < 225.0 - order by c_custkey, row_num""" - runTest.run_query( - bc, - drill, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) - queryId = "TEST_55" - query = """select last_value(c_custkey) over - ( - partition by c_nationkey - order by c_name - ) last_key, - c_phone, c_nationkey, UPPER(SUBSTRING(c_name, 1, 7)) - from customer - where c_acctbal < 225.0 - order by last_key, c_nationkey, c_phone""" - runTest.run_query( - bc, - spark, - query, - queryId, - queryType, - worder, - "", - acceptable_difference, - use_percentage, - fileSchemaType, - ) + # # using diffs columns to partition and first_value() + # queryId = "TEST_54" + # query = """select first_value(c_custkey) over + # ( + # partition by c_nationkey + # order by c_name desc + # ) row_num, + # c_phone, UPPER(SUBSTRING(c_name, 1, 7)) + # from customer + # where c_acctbal < 225.0 + # order by c_custkey, row_num""" + # runTest.run_query( + # bc, + # drill, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) + + # queryId = "TEST_55" + # query = """select last_value(c_custkey) over + # ( + # partition by c_nationkey + # order by c_name + # ) last_key, + # c_phone, c_nationkey, UPPER(SUBSTRING(c_name, 1, 7)) + # from customer + # where c_acctbal < 225.0 + # order by last_key, c_nationkey, c_phone""" + # runTest.run_query( + # bc, + # spark, + # query, + # queryId, + # queryType, + # worder, + # "", + # acceptable_difference, + # use_percentage, + # fileSchemaType, + # ) if Settings.execution_mode == ExecutionMode.GENERATOR: print("==============================") From 2c6d80b1c5a5366a37205d776d11f17ec34c8eb4 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Tue, 13 Apr 2021 17:41:17 -0500 Subject: [PATCH 03/18] some refactoring --- .../BatchWindowFunctionProcessing.cpp | 12 +++++------ .../logic_controllers/LogicalProject.cpp | 20 +++++++++---------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index f1e9301a0..47a507f4e 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -42,6 +42,12 @@ ComputeWindowKernel::ComputeWindowKernel(std::size_t kernel_id, const std::strin std::tie(this->column_indices_partitioned, std::ignore) = ral::operators::get_vars_to_partition(this->expression); std::tie(this->column_indices_ordered, std::ignore) = ral::operators::get_vars_to_orders(this->expression); + // fill all the Kind aggregations + for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { + AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); + this->aggs_wind_func.push_back(aggr_kind_i); + } + // if the window function has no partitioning but does have order by and a bounded window, then we need to remove the overlaps that are present in the data if (column_indices_partitioned.size() == 0 && column_indices_ordered.size() > 0 && this->preceding_value > 0 && this->following_value > 0){ this->remove_overlap = true; @@ -176,12 +182,6 @@ ral::execution::task_result ComputeWindowKernel::do_process(std::vector< std::un std::vector input_names = input->names(); - // fill all the Kind aggregations - for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { - AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); - this->aggs_wind_func.push_back(aggr_kind_i); - } - std::vector< std::unique_ptr > new_wf_cols; int agg_param_count = 0; for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { diff --git a/engine/src/execution_graph/logic_controllers/LogicalProject.cpp b/engine/src/execution_graph/logic_controllers/LogicalProject.cpp index 1d3cab704..31c91291a 100644 --- a/engine/src/execution_graph/logic_controllers/LogicalProject.cpp +++ b/engine/src/execution_graph/logic_controllers/LogicalProject.cpp @@ -787,11 +787,7 @@ std::vector> evaluate_expressions( const std::vector & expressions) { using interops::column_index_type; - // Let's clean all the expressions that contains Window functions (if exists) - // as they should be updated with new indices - std::vector new_expressions = clean_window_function_expressions(expressions, table.num_columns()); - - std::vector> out_columns(new_expressions.size()); + std::vector> out_columns(expressions.size()); std::vector column_used(table.num_columns(), false); std::vector> out_idx_computed_idx_pair; @@ -800,8 +796,8 @@ std::vector> evaluate_expressions( std::vector interpreter_out_column_views; function_evaluator_transformer evaluator{table}; - for(size_t i = 0; i < new_expressions.size(); i++){ - std::string expression = replace_calcite_regex(new_expressions[i]); + for(size_t i = 0; i < expressions.size(); i++){ + std::string expression = replace_calcite_regex(expressions[i]); expression = expand_if_logical_op(expression); parser::parse_tree tree; @@ -904,9 +900,9 @@ std::vector> evaluate_expressions( out_columns.clear(); computed_columns.clear(); - size_t const half_size = new_expressions.size() / 2; - std::vector split_lo(new_expressions.begin(), new_expressions.begin() + half_size); - std::vector split_hi(new_expressions.begin() + half_size, new_expressions.end()); + size_t const half_size = expressions.size() / 2; + std::vector split_lo(expressions.begin(), expressions.begin() + half_size); + std::vector split_hi(expressions.begin() + half_size, expressions.end()); auto out_cols_lo = evaluate_expressions(table, split_lo); auto out_cols_hi = evaluate_expressions(table, split_hi); @@ -955,6 +951,10 @@ std::unique_ptr process_project( out_column_names[i] = name; } + // Let's clean all the expressions that contains Window functions (if exists) + // as they should be updated with new indices + expressions = clean_window_function_expressions(expressions, blazing_table_in->num_columns()); + return std::make_unique(evaluate_expressions(blazing_table_in->view(), expressions), out_column_names); } From 33c111f71d1fb27c801dc2184651b6cd514348bd Mon Sep 17 00:00:00 2001 From: William Malpica Date: Wed, 14 Apr 2021 10:36:01 -0500 Subject: [PATCH 04/18] fixed get_bounds_from_window_expression and added more unit tests --- engine/src/parser/expression_utils.cpp | 36 +++++++++++++------ engine/tests/parser/expression_utils_test.cpp | 22 ++++++++---- 2 files changed, 41 insertions(+), 17 deletions(-) diff --git a/engine/src/parser/expression_utils.cpp b/engine/src/parser/expression_utils.cpp index 9f3a32846..9f4d033af 100644 --- a/engine/src/parser/expression_utils.cpp +++ b/engine/src/parser/expression_utils.cpp @@ -664,19 +664,35 @@ std::tuple< int, int > get_bounds_from_window_expression(const std::string & log // getting the first limit value std::string between_expr = "BETWEEN "; - std::string preceding_expr = " PRECEDING"; + std::string and_expr = "AND "; size_t start_pos = over_clause.find(between_expr) + between_expr.size(); - size_t end_pos = over_clause.find(preceding_expr); - std::string first_limit = over_clause.substr(start_pos, end_pos - start_pos); - preceding_value = std::stoi(first_limit); + size_t end_pos = over_clause.find(and_expr); + + std::string preceding_clause = over_clause.substr(start_pos, end_pos - start_pos); + std::string following_clause = over_clause.substr(end_pos + and_expr.size()); + + if (preceding_clause.find("CURRENT ROW") != std::string::npos){ + preceding_value = 0; + } else if (preceding_clause.find("UNBOUNDED") != std::string::npos){ + preceding_value = -1; + } else { + std::string preceding_expr = " PRECEDING"; + end_pos = preceding_clause.find(preceding_expr); + std::string str_value = preceding_clause.substr(0, end_pos); + preceding_value = std::stoi(str_value); + } // getting the second limit value - std::string and_expr = "AND "; - std::string following_expr = " FOLLOWING"; - start_pos = over_clause.find(and_expr) + and_expr.size(); - end_pos = over_clause.find(following_expr); - std::string second_limit = over_clause.substr(start_pos, end_pos - start_pos); - following_value = std::stoi(second_limit); + if (following_clause.find("CURRENT ROW") != std::string::npos){ + following_value = 0; + } else if (following_clause.find("UNBOUNDED") != std::string::npos){ + following_value = -1; + } else { + std::string following_expr = " FOLLOWING"; + end_pos = following_clause.find(following_expr); + std::string str_value = following_clause.substr(0, end_pos); + following_value = std::stoi(str_value); + } return std::make_tuple(preceding_value, following_value); } diff --git a/engine/tests/parser/expression_utils_test.cpp b/engine/tests/parser/expression_utils_test.cpp index 775bb62ec..354eaa149 100644 --- a/engine/tests/parser/expression_utils_test.cpp +++ b/engine/tests/parser/expression_utils_test.cpp @@ -111,13 +111,21 @@ TEST_F(ExpressionUtilsTest, gettings_bounds_from_window_expression) { EXPECT_EQ(preceding_value, expected_preceding); EXPECT_EQ(following_value, expected_following); - int expected_preceding2 = 1; - int expected_following2 = 2; - std::string query_part_2 = "max_keys=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 ROWS BETWEEN 1 PRECEDING AND 2 FOLLOWING)]"; - std::tie(preceding_value, following_value) = get_bounds_from_window_expression(query_part_2); - - EXPECT_EQ(preceding_value, expected_preceding2); - EXPECT_EQ(following_value, expected_following2); + std::vector preceding_options ={"UNBOUNDED PRECEDING", "4 PRECEDING", "CURRENT ROW"}; + std::vector following_options ={"UNBOUNDED FOLLOWING", "4 FOLLOWING", "CURRENT ROW"}; + std::vector expected = {-1, 4, 0}; + for (int i = 0; i < preceding_options.size(); ++i) { + for (int j = 0; j < following_options.size(); ++j) { + if (!(preceding_options[i] == "CURRENT ROW" && following_options[j] == "CURRENT ROW")){ + int expected_preceding2 = expected[i]; + int expected_following2 = expected[j]; + std::string query_part_2 = "max_keys=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 ROWS BETWEEN " + preceding_options[i] + " AND " + following_options[j] + ")]"; + std::tie(preceding_value, following_value) = get_bounds_from_window_expression(query_part_2); + EXPECT_EQ(preceding_value, expected_preceding2); + EXPECT_EQ(following_value, expected_following2); + } + } + } } TEST_F(ExpressionUtilsTest, getting_cols_to_apply_window_and_cols_to_apply_agg) { From 93e1fc976eddb10eb4fa6d953ae02d4f6d45cc28 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Wed, 14 Apr 2021 16:49:06 -0500 Subject: [PATCH 05/18] fixed issues with different types of windows. Added more e2e tests --- .../BatchWindowFunctionProcessing.cpp | 5 +- .../logic_controllers/PhysicalPlanGenerator.h | 8 +- .../EndToEndTests/windowFunctionTest.py | 182 ++++++++++++++++++ 3 files changed, 192 insertions(+), 3 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 47a507f4e..900176476 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -143,7 +143,10 @@ std::unique_ptr ComputeWindowKernel::compute_column_from_window_func else if (window_expression_contains_order_by(this->expression)) { if (window_expression_contains_bounds(this->expression)) { // TODO: for now just ROWS bounds works (not RANGE) - windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, this->preceding_value + 1, this->following_value, 1, window_aggregation); + windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, + this->preceding_value >= 0 ? this->preceding_value + 1: partitioned_table_view.num_rows(), + this->following_value >= 0 ? this->following_value : partitioned_table_view.num_rows(), + 1, window_aggregation); } else { if (this->type_aggs_as_str[pos] == "LEAD") { windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, 0, col_view_to_agg.size(), 1, window_aggregation); diff --git a/engine/src/execution_graph/logic_controllers/PhysicalPlanGenerator.h b/engine/src/execution_graph/logic_controllers/PhysicalPlanGenerator.h index 15d48ba18..9752e7dd1 100644 --- a/engine/src/execution_graph/logic_controllers/PhysicalPlanGenerator.h +++ b/engine/src/execution_graph/logic_controllers/PhysicalPlanGenerator.h @@ -293,12 +293,16 @@ struct tree_processor { } } else if (is_project(expr) && is_window_function(expr) && first_windowed_call) { + + // Calcite for some reason makes double UNBOUNDED windows always be set as RANGE. If we treat them as ROWS its easier and equivalent + StringUtil::findAndReplaceAll(expr, "RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", "ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING"); + if (window_expression_contains_multiple_diff_over_clauses(expr)) { - throw std::runtime_error("In Window Function: multiple WINDOW FUNCTIONs with different OVER clauses are not supported currently"); + throw std::runtime_error("In Window Function: multiple WINDOW FUNCTIONs with different OVER clauses are not supported currently. Expression found is: " + expr); } if (window_expression_contains_bounds_by_range(expr)) { - throw std::runtime_error("In Window Function: RANGE is not currently supported"); + throw std::runtime_error("In Window Function: RANGE is not currently supported. Expression found is: " + expr); } if (window_expression_contains_partition_by(expr)) { diff --git a/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py b/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py index e3f85724b..671b33905 100644 --- a/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py +++ b/tests/BlazingSQLTest/EndToEndTests/windowFunctionTest.py @@ -1255,6 +1255,188 @@ def executionTest(): fileSchemaType, ) + # testing different types of windows + queryId = "TEST_56" + query = """select o_orderkey, min(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN 5 PRECEDING + AND UNBOUNDED FOLLOWING + ) min_keys, + max(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN 5 PRECEDING + AND UNBOUNDED FOLLOWING + ) max_keys, o_custkey + from orders + where o_orderpriority <> '2-HIGH' + and o_clerk = 'Clerk#000000880' + and o_orderstatus is not null + and o_totalprice is not null + -- TODO using nulls last here because that is the default in BSQL but not in spark. + order by o_orderkey nulls last + limit 50""" + runTest.run_query( + bc, + spark, + query, + queryId, + queryType, + worder, + "", + acceptable_difference, + use_percentage, + fileSchemaType, + ) + + queryId = "TEST_57" + query = """select o_orderkey, min(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN UNBOUNDED PRECEDING + AND 5 FOLLOWING + ) min_keys, + max(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN UNBOUNDED PRECEDING + AND 5 FOLLOWING + ) max_keys, o_custkey + from orders + where o_orderpriority <> '2-HIGH' + and o_clerk = 'Clerk#000000880' + and o_orderstatus is not null + and o_totalprice is not null + -- TODO using nulls last here because that is the default in BSQL but not in spark. + order by o_orderkey nulls last + limit 50""" + runTest.run_query( + bc, + spark, + query, + queryId, + queryType, + worder, + "", + acceptable_difference, + use_percentage, + fileSchemaType, + ) + + queryId = "TEST_58" + query = """select o_orderkey, min(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN CURRENT ROW + AND UNBOUNDED FOLLOWING + ) min_keys, + max(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN CURRENT ROW + AND UNBOUNDED FOLLOWING + ) max_keys, o_custkey + from orders + where o_orderpriority <> '2-HIGH' + and o_clerk = 'Clerk#000000880' + and o_orderstatus is not null + and o_totalprice is not null + -- TODO using nulls last here because that is the default in BSQL but not in spark. + order by o_orderkey nulls last + limit 50""" + runTest.run_query( + bc, + spark, + query, + queryId, + queryType, + worder, + "", + acceptable_difference, + use_percentage, + fileSchemaType, + ) + + queryId = "TEST_59" + query = """select o_orderkey, min(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN UNBOUNDED PRECEDING + AND UNBOUNDED FOLLOWING + ) min_keys, + max(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN UNBOUNDED PRECEDING + AND UNBOUNDED FOLLOWING + ) max_keys, o_custkey + from orders + where o_orderpriority <> '2-HIGH' + and o_clerk = 'Clerk#000000880' + and o_orderstatus is not null + and o_totalprice is not null + -- TODO using nulls last here because that is the default in BSQL but not in spark. + order by o_orderkey nulls last + limit 50""" + runTest.run_query( + bc, + spark, + query, + queryId, + queryType, + worder, + "", + acceptable_difference, + use_percentage, + fileSchemaType, + ) + + + queryId = "TEST_60" + query = """select o_orderkey, min(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN 10 PRECEDING + AND CURRENT ROW + ) min_keys, + max(o_orderkey) over + ( + partition by o_custkey + order by o_totalprice + ROWS BETWEEN 10 PRECEDING + AND CURRENT ROW + ) max_keys, o_custkey + from orders + where o_orderpriority <> '2-HIGH' + and o_clerk = 'Clerk#000000880' + and o_orderstatus is not null + and o_totalprice is not null + -- TODO using nulls last here because that is the default in BSQL but not in spark. + order by o_orderkey nulls last + limit 50""" + runTest.run_query( + bc, + spark, + query, + queryId, + queryType, + worder, + "", + acceptable_difference, + use_percentage, + fileSchemaType, + ) + if Settings.execution_mode == ExecutionMode.GENERATOR: print("==============================") break From a5d1b73de25de81752cb3449d419e9c64cc18106 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Wed, 14 Apr 2021 17:23:14 -0500 Subject: [PATCH 06/18] updated CHANGELOG --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index bb7568a4e..98612368f 100755 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,7 +2,7 @@ ## New Features - +- #1471 Unbounded partitioned windows ## Improvements From d1bed0a7c66553549e79fd91d61feb10cad46a42 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Thu, 15 Apr 2021 10:54:19 -0500 Subject: [PATCH 07/18] did some refactoring. Also more window WIP --- .../BatchOrderByProcessing.cpp | 13 +--- .../BatchWindowFunctionProcessing.cpp | 78 ++++++++++++++++--- .../BatchWindowFunctionProcessing.h | 1 + .../logic_controllers/CacheData.cpp | 12 ++- .../logic_controllers/CacheMachine.cpp | 17 ++-- engine/src/io/DataLoader.cpp | 6 +- engine/src/utilities/CommonOperations.cpp | 8 ++ engine/src/utilities/CommonOperations.h | 1 + 8 files changed, 95 insertions(+), 41 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp index 6024822c8..6359dc667 100644 --- a/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp @@ -134,9 +134,9 @@ void SortAndSampleKernel::make_partition_plan_task(){ if(!(nodes[i] == ral::communication::CommunicationData::getInstance().getSelfNode())) { std::string message_id = std::to_string(this->context->getContextToken()) + "_" + std::to_string(this->get_id()) + "_" + nodes[i].id(); auto samples_cache_data = this->query_graph->get_input_message_cache()->pullCacheData(message_id); - ral::cache::CPUCacheData * cache_ptr = static_cast (samples_cache_data.get()); - total_num_rows_for_sampling += std::stoll(cache_ptr->getMetadata().get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]); - total_bytes_for_sampling += std::stoll(cache_ptr->getMetadata().get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]) * std::stoll(cache_ptr->getMetadata().get_values()[ral::cache::AVG_BYTES_PER_ROW_METADATA_LABEL]); + auto metadata = samples_cache_data->getMetadata(); + total_num_rows_for_sampling += std::stoll(metadata.get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]); + total_bytes_for_sampling += std::stoll(metadata.get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]) * std::stoll(metadata.get_values()[ral::cache::AVG_BYTES_PER_ROW_METADATA_LABEL]); sampleCacheDatas.push_back(std::move(samples_cache_data)); } } @@ -186,12 +186,7 @@ void SortAndSampleKernel::compute_partition_plan( } else { context->incrementQuerySubstep(); - // just to concat all the samples - std::vector sampledTableViews; - for (std::size_t i = 0; i < inputSamples.size(); i++){ - sampledTableViews.push_back(inputSamples[i]->toBlazingTableView()); - } - auto concatSamples = ral::utilities::concatTables(sampledTableViews); + auto concatSamples = ral::utilities::concatTables(std::move(inputSamples)); concatSamples->ensureOwnership(); ral::cache::MetadataDictionary extra_metadata; diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index ffbf600dd..fe8308b4e 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -1033,9 +1033,6 @@ ComputeWindowKernelUnbounded::ComputeWindowKernelUnbounded(std::size_t kernel_id } } -// WSM LEFT OFF HERE: lets make a full matrix of support and implementation and figure out the gaps -// want to include what aggs support what clauses - // TODO: Support for RANK() and DENSE_RANK() std::unique_ptr ComputeWindowKernelUnbounded::compute_column_from_window_function( cudf::table_view input_table_cudf_view, @@ -1120,10 +1117,13 @@ std::unique_ptr ComputeWindowKernelUnbounded::compute_column_from_wi windowed_col = std::move(sorted_table->release()[position_of_values_column]); } } - else if (window_expression_contains_order_by(this->expression)) { + else if (window_expression_contains_order_by(this->expression)) { if (window_expression_contains_bounds(this->expression)) { // TODO: for now just ROWS bounds works (not RANGE) - windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, this->preceding_value + 1, this->following_value, 1, window_aggregation); + windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, + this->preceding_value >= 0 ? this->preceding_value + 1: partitioned_table_view.num_rows(), + this->following_value >= 0 ? this->following_value : partitioned_table_view.num_rows(), + 1, window_aggregation); } else { if (this->type_aggs_as_str[pos] == "LEAD") { windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, 0, col_view_to_agg.size(), 1, window_aggregation); @@ -1238,11 +1238,11 @@ ral::execution::task_result ComputeWindowKernelUnbounded::do_process(std::vector cudf::table_view window_agg_table_view(window_agg_columns); cudf::table_view partial_aggregations_table_view; if (window_agg_table_view.num_rows() > 2){ - if (preceding is unbounded) { + if (this->preceding_value == -1) { // is unbounded // want to get last value std::vector split_window_view = cudf::split(window_agg_table_view, {window_agg_table_view.num_rows() - 2}); partial_aggregations_table_view = split_window_view[1]; - } else if (following is unbounded) { + } else if (this->following_value == -1) { // is unbounded // want to get first value std::vector split_window_view = cudf::split(window_agg_table_view, {1}); partial_aggregations_table_view = split_window_view[0]; @@ -1252,6 +1252,8 @@ ral::execution::task_result ComputeWindowKernelUnbounded::do_process(std::vector } else { partial_aggregations_table_view = window_agg_table_view; } + ral::frame::BlazingTableView partial_aggregations_BlazingTableView(partial_aggregations_table_view, window_agg_column_names); + partial_aggregations.push_back(std::move(partial_aggregations_BlazingTableView.clone())); // } @@ -1275,7 +1277,7 @@ kstatus ComputeWindowKernelUnbounded::run() { bool is_last_batch = false; bool is_first_node = self_node_idx == 0; bool is_last_node = self_node_idx == num_nodes - 1; - + std::unique_ptr cache_data = this->input_cache()->pullCacheData(); if (this->remove_overlap){ @@ -1290,7 +1292,7 @@ kstatus ComputeWindowKernelUnbounded::run() { std::map task_args; task_args[TASK_ARG_REMOVE_PRECEDING_OVERLAP] = !(is_first_batch && is_first_node) ? TRUE : FALSE; - task_args[TASK_ARG_REMOVE_FOLLOWING_OVERLAP] = !(is_last_batch && is_last_node) ? TRUE : FALSE; + task_args[TASK_ARG_REMOVE_FOLLOWING_OVERLAP] = !(is_last_batch && is_last_node) ? TRUE : FALSE; ral::execution::executor::get_instance()->add_task( std::move(inputs), @@ -1314,13 +1316,67 @@ kstatus ComputeWindowKernelUnbounded::run() { } } - - std::unique_lock lock(kernel_mutex); kernel_cv.wait(lock,[this]{ return this->tasks.empty(); }); + // WSM TODO we dont really want concatTables to be in the run function + std::unique_ptr partial_aggregations_table = ral::utilities::concatTables(std::move(this->partial_aggregations)); + partial_aggregations_table->ensureOwnership(); + + + // WSM TODO distributed vs single node???? + if(context->isMasterNode(ral::communication::CommunicationData::getInstance().getSelfNode())) { + auto nodes = context->getAllNodes(); + std::vector> all_partial_aggregations; + std::vector num_batches_per_node; + for(std::size_t i = 0; i < nodes.size(); ++i) { + if(!(nodes[i] == ral::communication::CommunicationData::getInstance().getSelfNode())) { + std::string message_id = std::to_string(this->context->getContextToken()) + "_" + std::to_string(this->get_id()) + "_" + nodes[i].id(); + auto partial_aggregations_cache_data = this->query_graph->get_input_message_cache()->pullCacheData(message_id); + num_batches_per_node.push_back(partial_aggregations_cache_data->num_rows()); + all_partial_aggregations.push_back(std::move(partial_aggregations_cache_data->decache())); + } else { + num_batches_per_node.push_back(partial_aggregations_table->num_rows()); + all_partial_aggregations.push_back(std::move(partial_aggregations_table)); + } + } + // WSM left off here. Perform cumulative aggregation and send it back + for (std::size_t col_i = 0; col_i < this->aggs_wind_func.size(); ++col_i) { + AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); + this->aggs_wind_func.push_back(aggr_kind_i); + } + if (this->preceding_value == -1) { // is unbounded + // want to get last value + std::vector split_window_view = cudf::split(window_agg_table_view, {window_agg_table_view.num_rows() - 2}); + partial_aggregations_table_view = split_window_view[1]; + } else if (this->following_value == -1) { // is unbounded + // want to get first value + std::vector split_window_view = cudf::split(window_agg_table_view, {1}); + partial_aggregations_table_view = split_window_view[0]; + } else { + // error one of the two should be unbounded otherwise we should not be in this kernel + } + +// https://github.com/rapidsai/cudf/issues/7967 + std::unique_ptr scan( + const column_view &input, + std::unique_ptr const &agg, + scan_type inclusive, + null_policy null_handling = null_policy::EXCLUDE, + rmm::mr::device_memory_resource *mr = rmm::mr::get_current_device_resource()); + } else { + send_message(std::move(partial_aggregations_table), + false, //specific_cache + "", //cache_id + {this->context->getMasterNode().id()}, //target_id + "", //message_id_prefix + true); //always_add + + } + + // WSM TODO here if is master node, wait until we get all partial_aggregations // Then combine partial_aggregations // Then send out cummulative_aggregations to nodes diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index 2976d5792..042c4564e 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -227,6 +227,7 @@ class ComputeWindowKernelUnbounded : public kernel { std::vector type_aggs_as_str; // ["MIN", "LAG"] std::vector aggs_wind_func; // [AggregateKind::MIN, AggregateKind::LAG] bool remove_overlap; // If we need to remove the overlaps after computing the windows + std::vector> partial_aggregations; // container to hold single row tables of the partial_aggregation values for each batch }; diff --git a/engine/src/execution_graph/logic_controllers/CacheData.cpp b/engine/src/execution_graph/logic_controllers/CacheData.cpp index c4ef8057e..afa4b16e2 100644 --- a/engine/src/execution_graph/logic_controllers/CacheData.cpp +++ b/engine/src/execution_graph/logic_controllers/CacheData.cpp @@ -307,15 +307,13 @@ std::unique_ptr ConcatCacheData::decache() { return _cache_datas[0]->decache(); } - std::vector> tables_holder; - std::vector table_views; + std::vector> tables; for (auto && cache_data : _cache_datas){ - tables_holder.push_back(cache_data->decache()); - table_views.push_back(tables_holder.back()->toBlazingTableView()); - - RAL_EXPECTS(!ral::utilities::checkIfConcatenatingStringsWillOverflow(table_views), "Concatenating tables will overflow"); + tables.push_back(cache_data->decache()); + + RAL_EXPECTS(!ral::utilities::checkIfConcatenatingStringsWillOverflow(tables), "Concatenating tables will overflow"); } - return ral::utilities::concatTables(table_views); + return ral::utilities::concatTables(std::move(tables)); } size_t ConcatCacheData::sizeInBytes() const { diff --git a/engine/src/execution_graph/logic_controllers/CacheMachine.cpp b/engine/src/execution_graph/logic_controllers/CacheMachine.cpp index d172c26da..7344d427e 100644 --- a/engine/src/execution_graph/logic_controllers/CacheMachine.cpp +++ b/engine/src/execution_graph/logic_controllers/CacheMachine.cpp @@ -782,21 +782,18 @@ std::unique_ptr ConcatenatingCacheMachine::pullFromCac output = data->decache(); num_rows = output->num_rows(); } else { - std::vector> tables_holder; - std::vector table_views; + std::vector> tables; for (size_t i = 0; i < collected_messages.size(); i++){ CodeTimer cacheEventTimer; cacheEventTimer.start(); auto data = collected_messages[i]->release_data(); - tables_holder.push_back(data->decache()); - table_views.push_back(tables_holder[i]->toBlazingTableView()); - + tables.push_back(data->decache()); + // if we dont have to concatenate all, lets make sure we are not overflowing, and if we are, lets put one back - if (!concat_all && ral::utilities::checkIfConcatenatingStringsWillOverflow(table_views)){ - auto cache_data = std::make_unique(std::move(tables_holder.back())); - tables_holder.pop_back(); - table_views.pop_back(); + if (!concat_all && ral::utilities::checkIfConcatenatingStringsWillOverflow(tables)){ + auto cache_data = std::make_unique(std::move(tables.back())); + tables.pop_back(); collected_messages[i] = std::make_unique(std::move(cache_data), collected_messages[i]->get_message_id()); for (; i < collected_messages.size(); i++){ this->waitingCache->put(std::move(collected_messages[i])); @@ -839,7 +836,7 @@ std::unique_ptr ConcatenatingCacheMachine::pullFromCac "description"_a="In ConcatenatingCacheMachine::pullFromCache Concatenating will overflow strings length"); } } - output = ral::utilities::concatTables(table_views); + output = ral::utilities::concatTables(std::move(tables)); num_rows = output->num_rows(); } diff --git a/engine/src/io/DataLoader.cpp b/engine/src/io/DataLoader.cpp index bd653cc67..2f4788c05 100644 --- a/engine/src/io/DataLoader.cpp +++ b/engine/src/io/DataLoader.cpp @@ -68,7 +68,6 @@ std::unique_ptr data_loader::get_metadata(int offset) std::size_t NUM_FILES_AT_A_TIME = 64; std::vector> metadata_batches; - std::vector metadata_batches_views; while(this->provider->has_next()){ std::vector> files; std::vector handles = this->provider->get_some(NUM_FILES_AT_A_TIME); @@ -76,7 +75,6 @@ std::unique_ptr data_loader::get_metadata(int offset) files.push_back(handle.file_handle); } metadata_batches.emplace_back(this->parser->get_metadata(handles, offset)); - metadata_batches_views.emplace_back(metadata_batches.back()->toBlazingTableView()); offset += files.size(); this->provider->close_file_handles(); } @@ -84,7 +82,7 @@ std::unique_ptr data_loader::get_metadata(int offset) if (metadata_batches.size() == 1){ return std::move(metadata_batches[0]); } else { - if(ral::utilities::checkIfConcatenatingStringsWillOverflow(metadata_batches_views)) { + if(ral::utilities::checkIfConcatenatingStringsWillOverflow(metadata_batches)) { std::shared_ptr logger = spdlog::get("batch_logger"); if(logger){ logger->warn("|||{info}|||||", @@ -92,7 +90,7 @@ std::unique_ptr data_loader::get_metadata(int offset) } } - return ral::utilities::concatTables(metadata_batches_views); + return ral::utilities::concatTables(std::move(metadata_batches)); } } diff --git a/engine/src/utilities/CommonOperations.cpp b/engine/src/utilities/CommonOperations.cpp index 4892cbcea..7e53463a8 100644 --- a/engine/src/utilities/CommonOperations.cpp +++ b/engine/src/utilities/CommonOperations.cpp @@ -77,6 +77,14 @@ bool checkIfConcatenatingStringsWillOverflow(const std::vector return false; } +std::unique_ptr concatTables(std::vector> tables){ + std::vector tables_views(tables.size()); + for (std::size_t i = 0; i < tables.size(); i++){ + tables_views[i] = tables[i]->toBlazingTableView(); + } + return concatTables(tables_views); +} + std::unique_ptr concatTables(const std::vector & tables) { assert(tables.size() >= 0); diff --git a/engine/src/utilities/CommonOperations.h b/engine/src/utilities/CommonOperations.h index 92578450f..dfc9069c6 100644 --- a/engine/src/utilities/CommonOperations.h +++ b/engine/src/utilities/CommonOperations.h @@ -13,6 +13,7 @@ using namespace ral::frame; bool checkIfConcatenatingStringsWillOverflow(const std::vector & tables); bool checkIfConcatenatingStringsWillOverflow(const std::vector> & tables); +std::unique_ptr concatTables(std::vector> tables); std::unique_ptr concatTables(const std::vector & tables); std::unique_ptr getLimitedRows(const BlazingTableView& table, cudf::size_type num_rows, bool front=true); From b6ef1772de2451c2418c46b3639f59bc61dbf056 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Thu, 15 Apr 2021 13:40:12 -0500 Subject: [PATCH 08/18] did some more refactoring and implementation --- .../BatchWindowFunctionProcessing.cpp | 36 +++++-------------- .../BatchWindowFunctionProcessing.h | 13 +++---- engine/src/parser/expression_utils.cpp | 13 +++++-- 3 files changed, 27 insertions(+), 35 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index fe8308b4e..9fd972276 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -60,19 +60,10 @@ ComputeWindowKernel::ComputeWindowKernel(std::size_t kernel_id, const std::strin std::unique_ptr ComputeWindowKernel::compute_column_from_window_function( cudf::table_view input_table_cudf_view, cudf::column_view col_view_to_agg, - std::size_t pos, int & agg_param_count ) { - - std::unique_ptr window_aggregation; + std::size_t pos ) { // we want firs get the type of aggregation - if (this->agg_param_values.size() > agg_param_count && is_lag_or_lead_aggregation(this->type_aggs_as_str[pos])) { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[agg_param_count]); - agg_param_count++; - } else if (is_last_value_window(this->type_aggs_as_str[pos])) { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], -1); - } else { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos]); - } + std::unique_ptr window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[pos]); // want all columns to be partitioned std::vector columns_to_partition; @@ -186,12 +177,11 @@ ral::execution::task_result ComputeWindowKernel::do_process(std::vector< std::un std::vector input_names = input->names(); std::vector< std::unique_ptr > new_wf_cols; - int agg_param_count = 0; for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { cudf::column_view col_view_to_agg = input_table_cudf_view.column(column_indices_to_agg[col_i]); // calling main window function - std::unique_ptr windowed_col = compute_column_from_window_function(input_table_cudf_view, col_view_to_agg, col_i, agg_param_count); + std::unique_ptr windowed_col = compute_column_from_window_function(input_table_cudf_view, col_view_to_agg, col_i); new_wf_cols.push_back(std::move(windowed_col)); } @@ -1037,19 +1027,12 @@ ComputeWindowKernelUnbounded::ComputeWindowKernelUnbounded(std::size_t kernel_id std::unique_ptr ComputeWindowKernelUnbounded::compute_column_from_window_function( cudf::table_view input_table_cudf_view, cudf::column_view col_view_to_agg, - std::size_t pos, int & agg_param_count ) { + std::size_t pos) { std::unique_ptr window_aggregation; // we want firs get the type of aggregation - if (this->agg_param_values.size() > agg_param_count && is_lag_or_lead_aggregation(this->type_aggs_as_str[pos])) { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[agg_param_count]); - agg_param_count++; - } else if (is_last_value_window(this->type_aggs_as_str[pos])) { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], -1); - } else { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos]); - } + std::unique_ptr window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[pos]); // want all columns to be partitioned std::vector columns_to_partition; @@ -1163,12 +1146,11 @@ ral::execution::task_result ComputeWindowKernelUnbounded::do_process(std::vector std::vector input_names = input->names(); std::vector< std::unique_ptr > new_wf_cols; - int agg_param_count = 0; for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { cudf::column_view col_view_to_agg = input_table_cudf_view.column(column_indices_to_agg[col_i]); // calling main window function - std::unique_ptr windowed_col = compute_column_from_window_function(input_table_cudf_view, col_view_to_agg, col_i, agg_param_count); + std::unique_ptr windowed_col = compute_column_from_window_function(input_table_cudf_view, col_view_to_agg, col_i); new_wf_cols.push_back(std::move(windowed_col)); } @@ -1321,7 +1303,7 @@ kstatus ComputeWindowKernelUnbounded::run() { return this->tasks.empty(); }); - // WSM TODO we dont really want concatTables to be in the run function + // WSM TODO we dont really want concatTables to be in the run function. This whole block of code below should probably be a task std::unique_ptr partial_aggregations_table = ral::utilities::concatTables(std::move(this->partial_aggregations)); partial_aggregations_table->ensureOwnership(); @@ -1344,8 +1326,8 @@ kstatus ComputeWindowKernelUnbounded::run() { } // WSM left off here. Perform cumulative aggregation and send it back for (std::size_t col_i = 0; col_i < this->aggs_wind_func.size(); ++col_i) { - AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); - this->aggs_wind_func.push_back(aggr_kind_i); + std::unique_ptr window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[col_i], this->agg_param_values[col_i]); + } if (this->preceding_value == -1) { // is unbounded // want to get last value diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index 042c4564e..19f851b78 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -28,7 +28,7 @@ class ComputeWindowKernel : public kernel { std::unique_ptr compute_column_from_window_function( cudf::table_view input_cudf_view, cudf::column_view input_col_view, - std::size_t pos, int & agg_param_count); + std::size_t pos); std::string kernel_name() { return "ComputeWindow";} @@ -43,7 +43,7 @@ class ComputeWindowKernel : public kernel { std::vector column_indices_partitioned; // column indices to be partitioned: [1] std::vector column_indices_ordered; // column indices to be ordered: [3] std::vector column_indices_to_agg; // column indices to be agg: [0, 0] - std::vector agg_param_values; // due to LAG or LEAD: [5] + std::vector agg_param_values; // due to LAG or LEAD: [0, 5] int preceding_value; // X PRECEDING int following_value; // Y FOLLOWING std::string frame_type; // ROWS or RANGE @@ -206,7 +206,7 @@ class ComputeWindowKernelUnbounded : public kernel { std::unique_ptr compute_column_from_window_function( cudf::table_view input_cudf_view, cudf::column_view input_col_view, - std::size_t pos, int & agg_param_count); + std::size_t pos); std::string kernel_name() { return "ComputeWindowUnbounded";} @@ -217,15 +217,16 @@ class ComputeWindowKernelUnbounded : public kernel { kstatus run() override; private: +// ComputeWindowKernelUnbounded(min_keys=[MIN($0) OVER (ORDER BY $3 DESC)], lag_col=[MAX($0) OVER (PARTITION BY $1)], n_name=[$2]) std::vector column_indices_partitioned; // column indices to be partitioned: [1] std::vector column_indices_ordered; // column indices to be ordered: [3] std::vector column_indices_to_agg; // column indices to be agg: [0, 0] - std::vector agg_param_values; // due to LAG or LEAD: [5] + std::vector agg_param_values; // due to LAG or LEAD: [0, 0] int preceding_value; // X PRECEDING int following_value; // Y FOLLOWING std::string frame_type; // ROWS or RANGE - std::vector type_aggs_as_str; // ["MIN", "LAG"] - std::vector aggs_wind_func; // [AggregateKind::MIN, AggregateKind::LAG] + std::vector type_aggs_as_str; // ["MIN", "MAX"] + std::vector aggs_wind_func; // [AggregateKind::MIN, AggregateKind::MAX] bool remove_overlap; // If we need to remove the overlaps after computing the windows std::vector> partial_aggregations; // container to hold single row tables of the partial_aggregation values for each batch }; diff --git a/engine/src/parser/expression_utils.cpp b/engine/src/parser/expression_utils.cpp index 9f4d033af..09784f53a 100644 --- a/engine/src/parser/expression_utils.cpp +++ b/engine/src/parser/expression_utils.cpp @@ -755,12 +755,12 @@ std::string get_first_over_expression_from_logical_plan(const std::string & logi // input: LogicalComputeWindow(min_keys=[MIN($0) OVER (PARTITION BY $2 ORDER BY $1)], // max_keys=[MAX(4) OVER (PARTITION BY $2 ORDER BY $1)], // lead_val=[LEAD($0, 3) OVER (PARTITION BY $2 ORDER BY $1)]) -// output: < [0, 4, 0], ["MIN", "MAX", "LEAD"], [3] > +// output: < [0, 4, 0], ["MIN", "MAX", "LEAD"], [0, 0, 3] > std::tuple< std::vector, std::vector, std::vector > get_cols_to_apply_window_and_cols_to_apply_agg(const std::string & logical_plan) { std::vector column_index; std::vector aggregations; - std::vector agg_param_values; + std::vector agg_param_values; // will be set to 0 when not actually used std::string query_part = get_query_part(logical_plan); std::vector project_expressions = get_expressions_from_expression_list(query_part); @@ -777,6 +777,7 @@ get_cols_to_apply_window_and_cols_to_apply_agg(const std::string & logical_plan) if (split_parts[0] == "ROW_NUMBER()") { aggregations.push_back(StringUtil::replace(split_parts[0], "()", "")); column_index.push_back(0); + agg_param_values.push_back(0); } else if (split_parts[0] == "LAG" || split_parts[0] == "LEAD") { // we need to get the constant values std::string right_express = StringUtil::replace(split_parts[1], ")", ""); @@ -790,10 +791,18 @@ get_cols_to_apply_window_and_cols_to_apply_agg(const std::string & logical_plan) std::string indice = split_parts[1].substr(0, split_parts[1].find(")")); column_index.push_back(std::stoi(indice)); column_index.push_back(std::stoi(indice)); + agg_param_values.push_back(0); + agg_param_values.push_back(0); + } else if (is_last_value_window(project_expressions[i])) { + aggregations.push_back(split_parts[0]); + std::string col_index = StringUtil::replace(split_parts[1], ")", ""); + column_index.push_back(std::stoi(col_index)); + agg_param_values.push_back(-1); } else { aggregations.push_back(split_parts[0]); std::string col_index = StringUtil::replace(split_parts[1], ")", ""); column_index.push_back(std::stoi(col_index)); + agg_param_values.push_back(0); } } } From fc0d8342323badc49d8690f992f518bd05a5a68c Mon Sep 17 00:00:00 2001 From: William Malpica Date: Thu, 15 Apr 2021 15:15:34 -0500 Subject: [PATCH 09/18] WIP --- .../BatchWindowFunctionProcessing.cpp | 64 ++++++++++++------- 1 file changed, 41 insertions(+), 23 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 9fd972276..73786428c 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -1310,44 +1311,61 @@ kstatus ComputeWindowKernelUnbounded::run() { // WSM TODO distributed vs single node???? if(context->isMasterNode(ral::communication::CommunicationData::getInstance().getSelfNode())) { + std::vector agg_column_names = partial_aggregations_table->names(); auto nodes = context->getAllNodes(); - std::vector> all_partial_aggregations; + std::vector> node_partial_aggregations; std::vector num_batches_per_node; for(std::size_t i = 0; i < nodes.size(); ++i) { if(!(nodes[i] == ral::communication::CommunicationData::getInstance().getSelfNode())) { std::string message_id = std::to_string(this->context->getContextToken()) + "_" + std::to_string(this->get_id()) + "_" + nodes[i].id(); auto partial_aggregations_cache_data = this->query_graph->get_input_message_cache()->pullCacheData(message_id); num_batches_per_node.push_back(partial_aggregations_cache_data->num_rows()); - all_partial_aggregations.push_back(std::move(partial_aggregations_cache_data->decache())); + node_partial_aggregations.push_back(std::move(partial_aggregations_cache_data->decache())); } else { num_batches_per_node.push_back(partial_aggregations_table->num_rows()); - all_partial_aggregations.push_back(std::move(partial_aggregations_table)); + node_partial_aggregations.push_back(std::move(partial_aggregations_table)); } } - // WSM left off here. Perform cumulative aggregation and send it back + + std::unique_ptr all_partial_aggregations_table = ral::utilities::concatTables(std::move(node_partial_aggregations)); + // Perform cumulative aggregation + std::vector> cumulative_agg_columns; + std::unique_ptr reverse_sequence; // if we need it, we only want to create it once, so I will declare it out here for (std::size_t col_i = 0; col_i < this->aggs_wind_func.size(); ++col_i) { std::unique_ptr window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[col_i], this->agg_param_values[col_i]); - - } - if (this->preceding_value == -1) { // is unbounded - // want to get last value - std::vector split_window_view = cudf::split(window_agg_table_view, {window_agg_table_view.num_rows() - 2}); - partial_aggregations_table_view = split_window_view[1]; - } else if (this->following_value == -1) { // is unbounded - // want to get first value - std::vector split_window_view = cudf::split(window_agg_table_view, {1}); - partial_aggregations_table_view = split_window_view[0]; - } else { - // error one of the two should be unbounded otherwise we should not be in this kernel + + cudf::column_view partial_agg_column_view = all_partial_aggregations_table->view().column(col_i); + std::unique_ptr cumulative_agg_column; + if (this->preceding_value == -1) { // is unbounded + cumulative_agg_column = cudf::scan(partial_agg_column_view, window_aggregation, cudf::scan_type::INCLUSIVE); + } + if (this->following_value == -1) { // is unbounded + if (this->preceding_value == -1){ // if we already had to do preceding unbounded, we want to use the output of that + partial_agg_column_view = cumulative_agg_column->view(); + } + // before we do the scan, we will want to do a reverse on the data + // right now there is no reverse API (https://github.com/rapidsai/cudf/issues/7967) so we need to do a sequense and gather + if (reverse_sequence == nullptr){ // if not initialized + reverse_sequence = cudf::sequence(partial_agg_column_view.num_rows(), partial_agg_column_view.num_rows() - 1, -1); } + cudf::table_view temp_table_view({partial_agg_column_view}); + std::unique_ptr reversed = cudf::gather(temp_table_view, reverse_sequence->view()); -// https://github.com/rapidsai/cudf/issues/7967 - std::unique_ptr scan( - const column_view &input, - std::unique_ptr const &agg, - scan_type inclusive, - null_policy null_handling = null_policy::EXCLUDE, - rmm::mr::device_memory_resource *mr = rmm::mr::get_current_device_resource()); + // now we do the inclusive scan on the reversed data. + cumulative_agg_column = cudf::scan(reversed->get_column(0)->view(), window_aggregation, cudf::scan_type::INCLUSIVE); + + // after we do the scan, we want to reverse the data back + temp_table_view = cudf::table_view({cumulative_agg_column->view()}); + reversed = cudf::gather(temp_table_view, reverse_sequence->view()); + std::vector> temp_released = reversed->release(); + cumulative_agg_column = std::move(temp_released[0]); + } + cumulative_agg_columns.push_back(std::move(cumulative_agg_column)); + } + std::unique_ptr cumulative_agg_table = std::make_unique( + std::make_unique(std::move(cumulative_agg_columns)), agg_column_names); + + } else { send_message(std::move(partial_aggregations_table), false, //specific_cache From a778b81f363e4fe6eca1be4cc88bc962a19a15ae Mon Sep 17 00:00:00 2001 From: William Malpica Date: Fri, 16 Apr 2021 16:28:14 -0500 Subject: [PATCH 10/18] various refactors --- .../BatchOrderByProcessing.cpp | 13 ++-- .../BatchWindowFunctionProcessing.cpp | 16 +---- .../BatchWindowFunctionProcessing.h | 4 +- .../logic_controllers/CacheData.cpp | 12 ++-- .../logic_controllers/CacheMachine.cpp | 17 ++--- engine/src/io/DataLoader.cpp | 6 +- engine/src/parser/expression_utils.cpp | 67 +++++++++++-------- engine/src/utilities/CommonOperations.cpp | 8 +++ engine/src/utilities/CommonOperations.h | 1 + 9 files changed, 70 insertions(+), 74 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp index 6024822c8..6359dc667 100644 --- a/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchOrderByProcessing.cpp @@ -134,9 +134,9 @@ void SortAndSampleKernel::make_partition_plan_task(){ if(!(nodes[i] == ral::communication::CommunicationData::getInstance().getSelfNode())) { std::string message_id = std::to_string(this->context->getContextToken()) + "_" + std::to_string(this->get_id()) + "_" + nodes[i].id(); auto samples_cache_data = this->query_graph->get_input_message_cache()->pullCacheData(message_id); - ral::cache::CPUCacheData * cache_ptr = static_cast (samples_cache_data.get()); - total_num_rows_for_sampling += std::stoll(cache_ptr->getMetadata().get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]); - total_bytes_for_sampling += std::stoll(cache_ptr->getMetadata().get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]) * std::stoll(cache_ptr->getMetadata().get_values()[ral::cache::AVG_BYTES_PER_ROW_METADATA_LABEL]); + auto metadata = samples_cache_data->getMetadata(); + total_num_rows_for_sampling += std::stoll(metadata.get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]); + total_bytes_for_sampling += std::stoll(metadata.get_values()[ral::cache::TOTAL_TABLE_ROWS_METADATA_LABEL]) * std::stoll(metadata.get_values()[ral::cache::AVG_BYTES_PER_ROW_METADATA_LABEL]); sampleCacheDatas.push_back(std::move(samples_cache_data)); } } @@ -186,12 +186,7 @@ void SortAndSampleKernel::compute_partition_plan( } else { context->incrementQuerySubstep(); - // just to concat all the samples - std::vector sampledTableViews; - for (std::size_t i = 0; i < inputSamples.size(); i++){ - sampledTableViews.push_back(inputSamples[i]->toBlazingTableView()); - } - auto concatSamples = ral::utilities::concatTables(sampledTableViews); + auto concatSamples = ral::utilities::concatTables(std::move(inputSamples)); concatSamples->ensureOwnership(); ral::cache::MetadataDictionary extra_metadata; diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 900176476..a24b00f9e 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -60,19 +60,10 @@ ComputeWindowKernel::ComputeWindowKernel(std::size_t kernel_id, const std::strin std::unique_ptr ComputeWindowKernel::compute_column_from_window_function( cudf::table_view input_table_cudf_view, cudf::column_view col_view_to_agg, - std::size_t pos, int & agg_param_count ) { - - std::unique_ptr window_aggregation; + std::size_t pos ) { // we want firs get the type of aggregation - if (this->agg_param_values.size() > agg_param_count && is_lag_or_lead_aggregation(this->type_aggs_as_str[pos])) { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[agg_param_count]); - agg_param_count++; - } else if (is_last_value_window(this->type_aggs_as_str[pos])) { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], -1); - } else { - window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos]); - } + std::unique_ptr window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[pos]); // want all columns to be partitioned std::vector columns_to_partition; @@ -186,12 +177,11 @@ ral::execution::task_result ComputeWindowKernel::do_process(std::vector< std::un std::vector input_names = input->names(); std::vector< std::unique_ptr > new_wf_cols; - int agg_param_count = 0; for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { cudf::column_view col_view_to_agg = input_table_cudf_view.column(column_indices_to_agg[col_i]); // calling main window function - std::unique_ptr windowed_col = compute_column_from_window_function(input_table_cudf_view, col_view_to_agg, col_i, agg_param_count); + std::unique_ptr windowed_col = compute_column_from_window_function(input_table_cudf_view, col_view_to_agg, col_i); new_wf_cols.push_back(std::move(windowed_col)); } diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index 29a7c065d..69dcb66d6 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -28,7 +28,7 @@ class ComputeWindowKernel : public kernel { std::unique_ptr compute_column_from_window_function( cudf::table_view input_cudf_view, cudf::column_view input_col_view, - std::size_t pos, int & agg_param_count); + std::size_t pos); std::string kernel_name() { return "ComputeWindow";} @@ -43,7 +43,7 @@ class ComputeWindowKernel : public kernel { std::vector column_indices_partitioned; // column indices to be partitioned: [1] std::vector column_indices_ordered; // column indices to be ordered: [3] std::vector column_indices_to_agg; // column indices to be agg: [0, 0] - std::vector agg_param_values; // due to LAG or LEAD: [5] + std::vector agg_param_values; // due to LAG or LEAD: [0, 5] int preceding_value; // X PRECEDING int following_value; // Y FOLLOWING std::string frame_type; // ROWS or RANGE diff --git a/engine/src/execution_graph/logic_controllers/CacheData.cpp b/engine/src/execution_graph/logic_controllers/CacheData.cpp index c4ef8057e..714f5d6d0 100644 --- a/engine/src/execution_graph/logic_controllers/CacheData.cpp +++ b/engine/src/execution_graph/logic_controllers/CacheData.cpp @@ -307,15 +307,13 @@ std::unique_ptr ConcatCacheData::decache() { return _cache_datas[0]->decache(); } - std::vector> tables_holder; - std::vector table_views; + std::vector> tables; for (auto && cache_data : _cache_datas){ - tables_holder.push_back(cache_data->decache()); - table_views.push_back(tables_holder.back()->toBlazingTableView()); - - RAL_EXPECTS(!ral::utilities::checkIfConcatenatingStringsWillOverflow(table_views), "Concatenating tables will overflow"); + tables.push_back(cache_data->decache()); + + RAL_EXPECTS(!ral::utilities::checkIfConcatenatingStringsWillOverflow(tables), "Concatenating tables will overflow"); } - return ral::utilities::concatTables(table_views); + return ral::utilities::concatTables(std::move(tables)); } size_t ConcatCacheData::sizeInBytes() const { diff --git a/engine/src/execution_graph/logic_controllers/CacheMachine.cpp b/engine/src/execution_graph/logic_controllers/CacheMachine.cpp index d172c26da..1a360fc41 100644 --- a/engine/src/execution_graph/logic_controllers/CacheMachine.cpp +++ b/engine/src/execution_graph/logic_controllers/CacheMachine.cpp @@ -782,21 +782,18 @@ std::unique_ptr ConcatenatingCacheMachine::pullFromCac output = data->decache(); num_rows = output->num_rows(); } else { - std::vector> tables_holder; - std::vector table_views; + std::vector> tables; for (size_t i = 0; i < collected_messages.size(); i++){ CodeTimer cacheEventTimer; cacheEventTimer.start(); auto data = collected_messages[i]->release_data(); - tables_holder.push_back(data->decache()); - table_views.push_back(tables_holder[i]->toBlazingTableView()); + tables.push_back(data->decache()); // if we dont have to concatenate all, lets make sure we are not overflowing, and if we are, lets put one back - if (!concat_all && ral::utilities::checkIfConcatenatingStringsWillOverflow(table_views)){ - auto cache_data = std::make_unique(std::move(tables_holder.back())); - tables_holder.pop_back(); - table_views.pop_back(); + if (!concat_all && ral::utilities::checkIfConcatenatingStringsWillOverflow(tables)){ + auto cache_data = std::make_unique(std::move(tables.back())); + tables.pop_back(); collected_messages[i] = std::make_unique(std::move(cache_data), collected_messages[i]->get_message_id()); for (; i < collected_messages.size(); i++){ this->waitingCache->put(std::move(collected_messages[i])); @@ -821,7 +818,7 @@ std::unique_ptr ConcatenatingCacheMachine::pullFromCac } } - if( concat_all && ral::utilities::checkIfConcatenatingStringsWillOverflow(table_views) ) { // if we have to concatenate all, then lets throw a warning if it will overflow strings + if( concat_all && ral::utilities::checkIfConcatenatingStringsWillOverflow(tables) ) { // if we have to concatenate all, then lets throw a warning if it will overflow strings CodeTimer cacheEventTimer; cacheEventTimer.start(); cacheEventTimer.stop(); @@ -839,7 +836,7 @@ std::unique_ptr ConcatenatingCacheMachine::pullFromCac "description"_a="In ConcatenatingCacheMachine::pullFromCache Concatenating will overflow strings length"); } } - output = ral::utilities::concatTables(table_views); + output = ral::utilities::concatTables(std::move(tables)); num_rows = output->num_rows(); } diff --git a/engine/src/io/DataLoader.cpp b/engine/src/io/DataLoader.cpp index bd653cc67..2f4788c05 100644 --- a/engine/src/io/DataLoader.cpp +++ b/engine/src/io/DataLoader.cpp @@ -68,7 +68,6 @@ std::unique_ptr data_loader::get_metadata(int offset) std::size_t NUM_FILES_AT_A_TIME = 64; std::vector> metadata_batches; - std::vector metadata_batches_views; while(this->provider->has_next()){ std::vector> files; std::vector handles = this->provider->get_some(NUM_FILES_AT_A_TIME); @@ -76,7 +75,6 @@ std::unique_ptr data_loader::get_metadata(int offset) files.push_back(handle.file_handle); } metadata_batches.emplace_back(this->parser->get_metadata(handles, offset)); - metadata_batches_views.emplace_back(metadata_batches.back()->toBlazingTableView()); offset += files.size(); this->provider->close_file_handles(); } @@ -84,7 +82,7 @@ std::unique_ptr data_loader::get_metadata(int offset) if (metadata_batches.size() == 1){ return std::move(metadata_batches[0]); } else { - if(ral::utilities::checkIfConcatenatingStringsWillOverflow(metadata_batches_views)) { + if(ral::utilities::checkIfConcatenatingStringsWillOverflow(metadata_batches)) { std::shared_ptr logger = spdlog::get("batch_logger"); if(logger){ logger->warn("|||{info}|||||", @@ -92,7 +90,7 @@ std::unique_ptr data_loader::get_metadata(int offset) } } - return ral::utilities::concatTables(metadata_batches_views); + return ral::utilities::concatTables(std::move(metadata_batches)); } } diff --git a/engine/src/parser/expression_utils.cpp b/engine/src/parser/expression_utils.cpp index 9f4d033af..62d40077c 100644 --- a/engine/src/parser/expression_utils.cpp +++ b/engine/src/parser/expression_utils.cpp @@ -755,48 +755,57 @@ std::string get_first_over_expression_from_logical_plan(const std::string & logi // input: LogicalComputeWindow(min_keys=[MIN($0) OVER (PARTITION BY $2 ORDER BY $1)], // max_keys=[MAX(4) OVER (PARTITION BY $2 ORDER BY $1)], // lead_val=[LEAD($0, 3) OVER (PARTITION BY $2 ORDER BY $1)]) -// output: < [0, 4, 0], ["MIN", "MAX", "LEAD"], [3] > +// output: < [0, 4, 0], ["MIN", "MAX", "LEAD"], [0, 0, 3] > std::tuple< std::vector, std::vector, std::vector > get_cols_to_apply_window_and_cols_to_apply_agg(const std::string & logical_plan) { std::vector column_index; std::vector aggregations; - std::vector agg_param_values; + std::vector agg_param_values; // will be set to 0 when not actually used std::string query_part = get_query_part(logical_plan); std::vector project_expressions = get_expressions_from_expression_list(query_part); // we want all expressions that contains an OVER clause for (size_t i = 0; i < project_expressions.size(); ++i) { - if (project_expressions[i].find("OVER") != std::string::npos) { - std::string express_i = project_expressions[i]; - size_t start_pos = express_i.find("[") + 1; - size_t end_pos = express_i.find("OVER"); - express_i = express_i.substr(start_pos, end_pos - start_pos); - std::string express_i_wo_trim = StringUtil::trim(express_i); - std::vector split_parts = StringUtil::split(express_i_wo_trim, "($"); - if (split_parts[0] == "ROW_NUMBER()") { - aggregations.push_back(StringUtil::replace(split_parts[0], "()", "")); - column_index.push_back(0); - } else if (split_parts[0] == "LAG" || split_parts[0] == "LEAD") { - // we need to get the constant values - std::string right_express = StringUtil::replace(split_parts[1], ")", ""); - std::vector inside_parts = StringUtil::split(right_express, ", "); - aggregations.push_back(split_parts[0]); - column_index.push_back(std::stoi(inside_parts[0])); - agg_param_values.push_back(std::stoi(inside_parts[1])); - } else if ( is_sum_window_function(project_expressions[i]) || is_avg_window_function(project_expressions[i]) ) { - aggregations.push_back("COUNT"); - aggregations.push_back("$SUM0"); - std::string indice = split_parts[1].substr(0, split_parts[1].find(")")); - column_index.push_back(std::stoi(indice)); - column_index.push_back(std::stoi(indice)); - } else { + if (project_expressions[i].find("OVER") != std::string::npos) { + std::string express_i = project_expressions[i]; + size_t start_pos = express_i.find("[") + 1; + size_t end_pos = express_i.find("OVER"); + express_i = express_i.substr(start_pos, end_pos - start_pos); + std::string express_i_wo_trim = StringUtil::trim(express_i); + std::vector split_parts = StringUtil::split(express_i_wo_trim, "($"); + if (split_parts[0] == "ROW_NUMBER()") { + aggregations.push_back(StringUtil::replace(split_parts[0], "()", "")); + column_index.push_back(0); + agg_param_values.push_back(0); + } else if (split_parts[0] == "LAG" || split_parts[0] == "LEAD") { + // we need to get the constant values + std::string right_express = StringUtil::replace(split_parts[1], ")", ""); + std::vector inside_parts = StringUtil::split(right_express, ", "); + aggregations.push_back(split_parts[0]); + column_index.push_back(std::stoi(inside_parts[0])); + agg_param_values.push_back(std::stoi(inside_parts[1])); + } else if ( is_sum_window_function(project_expressions[i]) || is_avg_window_function(project_expressions[i]) ) { + aggregations.push_back("COUNT"); + aggregations.push_back("$SUM0"); + std::string indice = split_parts[1].substr(0, split_parts[1].find(")")); + column_index.push_back(std::stoi(indice)); + column_index.push_back(std::stoi(indice)); + agg_param_values.push_back(0); + agg_param_values.push_back(0); + } else if (is_last_value_window(project_expressions[i])) { aggregations.push_back(split_parts[0]); std::string col_index = StringUtil::replace(split_parts[1], ")", ""); column_index.push_back(std::stoi(col_index)); - } - } - } + agg_param_values.push_back(-1); + } else { + aggregations.push_back(split_parts[0]); + std::string col_index = StringUtil::replace(split_parts[1], ")", ""); + column_index.push_back(std::stoi(col_index)); + agg_param_values.push_back(0); + } + } + } return std::make_tuple(column_index, aggregations, agg_param_values); } diff --git a/engine/src/utilities/CommonOperations.cpp b/engine/src/utilities/CommonOperations.cpp index 4892cbcea..7e53463a8 100644 --- a/engine/src/utilities/CommonOperations.cpp +++ b/engine/src/utilities/CommonOperations.cpp @@ -77,6 +77,14 @@ bool checkIfConcatenatingStringsWillOverflow(const std::vector return false; } +std::unique_ptr concatTables(std::vector> tables){ + std::vector tables_views(tables.size()); + for (std::size_t i = 0; i < tables.size(); i++){ + tables_views[i] = tables[i]->toBlazingTableView(); + } + return concatTables(tables_views); +} + std::unique_ptr concatTables(const std::vector & tables) { assert(tables.size() >= 0); diff --git a/engine/src/utilities/CommonOperations.h b/engine/src/utilities/CommonOperations.h index 92578450f..dfc9069c6 100644 --- a/engine/src/utilities/CommonOperations.h +++ b/engine/src/utilities/CommonOperations.h @@ -13,6 +13,7 @@ using namespace ral::frame; bool checkIfConcatenatingStringsWillOverflow(const std::vector & tables); bool checkIfConcatenatingStringsWillOverflow(const std::vector> & tables); +std::unique_ptr concatTables(std::vector> tables); std::unique_ptr concatTables(const std::vector & tables); std::unique_ptr getLimitedRows(const BlazingTableView& table, cudf::size_type num_rows, bool front=true); From ffc161cb99049912ca241a9ce9afe3c4545a2ab2 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Mon, 19 Apr 2021 14:11:18 -0500 Subject: [PATCH 11/18] fixed build bugs added a little more --- .../BatchWindowFunctionProcessing.cpp | 21 ++++++++++++------- .../BatchWindowFunctionProcessing.h | 2 +- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 73786428c..2fe997aae 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -15,7 +15,7 @@ #include #include "cudf/column/column_view.hpp" #include -#include +#include #include #include #include @@ -1000,7 +1000,7 @@ In the future, when we can have CacheData's shared between nodes, then we can re ComputeWindowKernelUnbounded::ComputeWindowKernelUnbounded(std::size_t kernel_id, const std::string & queryString, std::shared_ptr context, std::shared_ptr query_graph) - : kernel{kernel_id, queryString, context, kernel_type::ComputeWindowKernel} { + : distributing_kernel{kernel_id, queryString, context, kernel_type::ComputeWindowKernel} { this->query_graph = query_graph; std::tie(this->preceding_value, this->following_value) = get_bounds_from_window_expression(this->expression); this->frame_type = get_frame_type_from_over_clause(this->expression); @@ -1030,8 +1030,6 @@ std::unique_ptr ComputeWindowKernelUnbounded::compute_column_from_wi cudf::column_view col_view_to_agg, std::size_t pos) { - std::unique_ptr window_aggregation; - // we want firs get the type of aggregation std::unique_ptr window_aggregation = ral::operators::makeCudfAggregation(this->aggs_wind_func[pos], this->agg_param_values[pos]); @@ -1314,7 +1312,7 @@ kstatus ComputeWindowKernelUnbounded::run() { std::vector agg_column_names = partial_aggregations_table->names(); auto nodes = context->getAllNodes(); std::vector> node_partial_aggregations; - std::vector num_batches_per_node; + std::vector num_batches_per_node; for(std::size_t i = 0; i < nodes.size(); ++i) { if(!(nodes[i] == ral::communication::CommunicationData::getInstance().getSelfNode())) { std::string message_id = std::to_string(this->context->getContextToken()) + "_" + std::to_string(this->get_id()) + "_" + nodes[i].id(); @@ -1346,24 +1344,31 @@ kstatus ComputeWindowKernelUnbounded::run() { // before we do the scan, we will want to do a reverse on the data // right now there is no reverse API (https://github.com/rapidsai/cudf/issues/7967) so we need to do a sequense and gather if (reverse_sequence == nullptr){ // if not initialized - reverse_sequence = cudf::sequence(partial_agg_column_view.num_rows(), partial_agg_column_view.num_rows() - 1, -1); + reverse_sequence = cudf::sequence(all_partial_aggregations_table->num_rows(), + cudf::numeric_scalar(all_partial_aggregations_table->num_rows() - 1, true), + cudf::numeric_scalar(-1, true)); } + cudf::table_view temp_table_view({partial_agg_column_view}); std::unique_ptr reversed = cudf::gather(temp_table_view, reverse_sequence->view()); // now we do the inclusive scan on the reversed data. - cumulative_agg_column = cudf::scan(reversed->get_column(0)->view(), window_aggregation, cudf::scan_type::INCLUSIVE); + cumulative_agg_column = cudf::scan(reversed->get_column(0).view(), window_aggregation, cudf::scan_type::INCLUSIVE); // after we do the scan, we want to reverse the data back temp_table_view = cudf::table_view({cumulative_agg_column->view()}); reversed = cudf::gather(temp_table_view, reverse_sequence->view()); - std::vector> temp_released = reversed->release(); + std::vector> temp_released = reversed->release(); cumulative_agg_column = std::move(temp_released[0]); } cumulative_agg_columns.push_back(std::move(cumulative_agg_column)); } std::unique_ptr cumulative_agg_table = std::make_unique( std::make_unique(std::move(cumulative_agg_columns)), agg_column_names); + + std::vector split_indexes(num_batches_per_node.size() - 1); + std::partial_sum(num_batches_per_node.begin(), num_batches_per_node.end()-1, split_indexes.begin()); + std::vector cumulative_agg_per_node = cudf::split(cumulative_agg_table->view(), split_indexes); } else { diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index 19f851b78..a9d8495f3 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -197,7 +197,7 @@ class OverlapAccumulatorKernel : public distributing_kernel { * New columns will be added to each batch */ -class ComputeWindowKernelUnbounded : public kernel { +class ComputeWindowKernelUnbounded : public distributing_kernel { public: ComputeWindowKernelUnbounded(std::size_t kernel_id, const std::string & queryString, std::shared_ptr context, From 1418be6c553cfe7832e1e05e177919cdfc1c0b7b Mon Sep 17 00:00:00 2001 From: William Malpica Date: Mon, 19 Apr 2021 21:03:35 -0500 Subject: [PATCH 12/18] working on WindowAggMergerKernel --- .../BatchWindowFunctionProcessing.cpp | 115 +++++++++++++++++- .../BatchWindowFunctionProcessing.h | 63 +++++++++- .../logic_controllers/taskflow/kernel.cpp | 4 +- .../logic_controllers/taskflow/kernel.h | 14 ++- 4 files changed, 178 insertions(+), 18 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 2fe997aae..5f7f32358 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -1022,6 +1022,10 @@ ComputeWindowKernelUnbounded::ComputeWindowKernelUnbounded(std::size_t kernel_id } else { this->remove_overlap = false; } + + // batches is for the regular computed window function results + // cumulative_aggregations is the cumulative aggregation values + this->output_.add_port("batches", "cumulative_aggregations"); } // TODO: Support for RANK() and DENSE_RANK() @@ -1369,7 +1373,18 @@ kstatus ComputeWindowKernelUnbounded::run() { std::vector split_indexes(num_batches_per_node.size() - 1); std::partial_sum(num_batches_per_node.begin(), num_batches_per_node.end()-1, split_indexes.begin()); std::vector cumulative_agg_per_node = cudf::split(cumulative_agg_table->view(), split_indexes); - + + std::vector bsql_cumulative_agg_per_node; + for(auto cumulative_agg : cumulative_agg_per_node) { + bsql_cumulative_agg_per_node.push_back(ral::frame::BlazingTableView(cumulative_agg, agg_column_names)); + } + + scatter(bsql_cumulative_agg_per_node, + this->output_.get_cache("cumulative_aggregations").get(), + "", //message_id_prefix + "cumulative_aggregations" //cache_id + ); + } else { send_message(std::move(partial_aggregations_table), @@ -1381,25 +1396,113 @@ kstatus ComputeWindowKernelUnbounded::run() { } + if (logger != nullptr) { + logger->debug("{query_id}|{step}|{substep}|{info}|{duration}|kernel_id|{kernel_id}||", + "query_id"_a=context->getContextToken(), + "step"_a=context->getQueryStep(), + "substep"_a=context->getQuerySubstep(), + "info"_a="ComputeWindowKernelUnbounded Kernel Completed", + "duration"_a=timer.elapsed_time(), + "kernel_id"_a=this->get_id()); + } + return kstatus::proceed; +} + +// END ComputeWindowKernelUnbounded + + +// BEGIN WindowAggMergerKernel + +WindowAggMergerKernel::WindowAggMergerKernel(std::size_t kernel_id, const std::string & queryString, + std::shared_ptr context, + std::shared_ptr query_graph) + : kernel{kernel_id, queryString, context, kernel_type::ComputeWindowKernel} { + this->query_graph = query_graph; + std::tie(this->preceding_value, this->following_value) = get_bounds_from_window_expression(this->expression); + this->frame_type = get_frame_type_from_over_clause(this->expression); + + std::tie(this->column_indices_to_agg, this->type_aggs_as_str, this->agg_param_values) = + get_cols_to_apply_window_and_cols_to_apply_agg(this->expression); + std::tie(this->column_indices_partitioned, std::ignore) = ral::operators::get_vars_to_partition(this->expression); + std::tie(this->column_indices_ordered, std::ignore) = ral::operators::get_vars_to_orders(this->expression); + + // fill all the Kind aggregations + for (std::size_t col_i = 0; col_i < this->type_aggs_as_str.size(); ++col_i) { + AggregateKind aggr_kind_i = ral::operators::get_aggregation_operation(this->type_aggs_as_str[col_i], true); + this->aggs_wind_func.push_back(aggr_kind_i); + } + + // batches is for the regular computed window function results + // cumulative_aggregations is the cumulative aggregation values + this->input_.add_port("batches", "cumulative_aggregations"); +} + + +ral::execution::task_result WindowAggMergerKernel::do_process(std::vector< std::unique_ptr > inputs, + std::shared_ptr output, + cudaStream_t /*stream*/, const std::map& args) { + + + if (inputs.size() == 0) { + return {ral::execution::task_status::SUCCESS, std::string(), std::vector< std::unique_ptr > ()}; + } + + std::unique_ptr & input = inputs[0]; + + try{ + + output->addToCache(std::move(windowed_table)); + }catch(const rmm::bad_alloc& e){ + return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; + }catch(const std::exception& e){ + return {ral::execution::task_status::FAIL, std::string(e.what()), std::vector< std::unique_ptr > ()}; + } + + return {ral::execution::task_status::SUCCESS, std::string(), std::vector< std::unique_ptr > ()}; +} + +kstatus WindowAggMergerKernel::run() { + CodeTimer timer; - // WSM TODO here if is master node, wait until we get all partial_aggregations - // Then combine partial_aggregations - // Then send out cummulative_aggregations to nodes + int self_node_idx = context->getNodeIndex(ral::communication::CommunicationData::getInstance().getSelfNode()); + int num_nodes = context->getTotalNodes(); + + + cumulative_aggregations = this->input_cache("cumulative_aggregations")->pullFromCache(); + std::unique_ptr cache_data = this->input_cache("batches")->pullCacheData(); + + while (cache_data != nullptr ){ + std::vector > inputs; + inputs.push_back(std::move(cache_data)); + + ral::execution::executor::get_instance()->add_task( + std::move(inputs), + this->output_cache(), + this, + task_args); + + cache_data = this->input_cache()->pullCacheData(); + } + + std::unique_lock lock(kernel_mutex); + kernel_cv.wait(lock,[this]{ + return this->tasks.empty(); + }); if (logger != nullptr) { logger->debug("{query_id}|{step}|{substep}|{info}|{duration}|kernel_id|{kernel_id}||", "query_id"_a=context->getContextToken(), "step"_a=context->getQueryStep(), "substep"_a=context->getQuerySubstep(), - "info"_a="ComputeWindow Kernel Completed", + "info"_a="WindowAggMergerKernel Kernel Completed", "duration"_a=timer.elapsed_time(), "kernel_id"_a=this->get_id()); } return kstatus::proceed; } -// END ComputeWindowKernelUnbounded +// END WindowAggMergerKernel } // namespace batch } // namespace ral diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index a9d8495f3..2b7d9ff0e 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -79,7 +79,7 @@ const std::string FOLLOWING_RESPONSE="following_response"; /** -* The OverlapGeneratorKernel is only used for window functions that have no partition by clause and that also have bounded window frames. +* @brief The OverlapGeneratorKernel is only used for window functions that have no partition by clause and that also have bounded window frames. * The OverlapGeneratorKernel assumes that it will be following by OverlapAccumulatorKernel and has three output caches: * - "batches" * - "preceding_overlaps" @@ -118,7 +118,7 @@ class OverlapGeneratorKernel : public kernel { /** -* The OverlapAccumulatorKernel assumes three input caches: +* @brief The OverlapAccumulatorKernel assumes three input caches: * - "batches" * - "preceding_overlaps" * - "following_overlaps" @@ -192,10 +192,16 @@ class OverlapAccumulatorKernel : public distributing_kernel { /** - * @brief This kernel computes the main Window Function (ROW_NUMBER, LAG, LEAD, MIN, ...) - * to each batch already pattitioned and sorted - * New columns will be added to each batch - */ +* @brief The ComputeWindowKernelUnbounded is only used for window functions that have no partition by clause and that also have unbounded window frames. +* The ComputeWindowKernelUnbounded assumes that it will be followed by WindowAggMergerKernel and has two output caches: +* - "batches" +* - "cumulative_aggregations" +* +* Its purpose is to take is to compute the main Window Function (ROW_NUMBER, LAG, LEAD, MIN, ...) to each batch +* which has already pattitioned and sorted, much like ComputeWindowKernel. Additionally it will calculate partial aggregation values and send them all to the +* mater node. The master node will accumulate the partial aggregation values and calculate cumulative aggregation values for the whole cluster. +* Then the master node will send back out to all other nodes the cumulative aggregation values that correspond to each node and add them to the cumulative_aggregations CacheMachine. +*/ class ComputeWindowKernelUnbounded : public distributing_kernel { public: @@ -232,6 +238,51 @@ class ComputeWindowKernelUnbounded : public distributing_kernel { }; + +/** +* @brief The WindowAggMergerKernel is only used for window functions that have no partition by clause and that also have unbounded window frames. +* The WindowAggMergerKernel assumes that it was preceded by ComputeWindowKernelUnbounded and has two input caches: +* - "batches" +* - "cumulative_aggregations" +* +* Its purpose is to take the cumulative_aggregations and merge them with each corresponding batch. +*/ +class WindowAggMergerKernel : public kernel { +public: + WindowAggMergerKernel(std::size_t kernel_id, const std::string & queryString, + std::shared_ptr context, + std::shared_ptr query_graph); + + std::string kernel_name() { return "WindowAggMerger";} + + ral::execution::task_result do_process(std::vector< std::unique_ptr > inputs, + std::shared_ptr output, + cudaStream_t stream, const std::map& args) override; + + kstatus run() override; + +private: + // WindowAggMergerKernel(min_keys=[MIN($0) OVER (ORDER BY $3 DESC)], lag_col=[MAX($0) OVER (PARTITION BY $1)], n_name=[$2]) + std::vector column_indices_partitioned; // column indices to be partitioned: [1] + std::vector column_indices_ordered; // column indices to be ordered: [3] + std::vector column_indices_to_agg; // column indices to be agg: [0, 0] + std::vector agg_param_values; // due to LAG or LEAD: [0, 0] + int preceding_value; // X PRECEDING + int following_value; // Y FOLLOWING + std::string frame_type; // ROWS or RANGE + std::vector type_aggs_as_str; // ["MIN", "MAX"] + std::vector aggs_wind_func; // [AggregateKind::MIN, AggregateKind::MAX] + + + // WSM TODO. Do we really want this to be a BlazingTable member? + // It should always be pretty small, so it should not be a memory burden of any note, but it goes against the pattern + // We can always figure out a host representation that we would be more comfortable being left as a member + std::unique_ptr cumulative_aggregations; + +}; + + + } // namespace batch } // namespace ral diff --git a/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp b/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp index ff151b8c4..16dad5a38 100644 --- a/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp +++ b/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp @@ -34,8 +34,8 @@ std::shared_ptr kernel::output_cache(std::string cache return this->output_.get_cache(cache_id); } -std::shared_ptr kernel::input_cache() { - auto kernel_id = std::to_string(this->get_id()); +std::shared_ptr kernel::input_cache(std::string cache_id) { + cache_id = cache_id.empty() ? std::to_string(this->get_id()) : cache_id; return this->input_.get_cache(kernel_id); } diff --git a/engine/src/execution_graph/logic_controllers/taskflow/kernel.h b/engine/src/execution_graph/logic_controllers/taskflow/kernel.h index e07a21899..852129045 100644 --- a/engine/src/execution_graph/logic_controllers/taskflow/kernel.h +++ b/engine/src/execution_graph/logic_controllers/taskflow/kernel.h @@ -96,14 +96,20 @@ class kernel { void set_type_id(kernel_type kernel_type_id_) { kernel_type_id = kernel_type_id_; } /** - * @brief Returns the input cache. + * @brief Returns the input cache associated to an identifier. If none is provided it returns the default + * + * @param cache_id The cache identifier. + * + * @return A shared pointer to the desired CacheMachine */ - std::shared_ptr input_cache(); + std::shared_ptr input_cache(std::string cache_id = ""); /** - * @brief Returns the output cache associated to an identifier. + * @brief Returns the output cache associated to an identifier. If none is provided it returns the default + * + * @param cache_id The cache identifier. * - * @return cache_id The identifier of the output cache. + * @return A shared pointer to the desired CacheMachine */ std::shared_ptr output_cache(std::string cache_id = ""); From e87e481952a1e57e2f3f428d391b468357da32d3 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Tue, 20 Apr 2021 20:28:50 -0500 Subject: [PATCH 13/18] fixed some build bugs --- .../logic_controllers/BatchWindowFunctionProcessing.cpp | 5 ++--- .../execution_graph/logic_controllers/taskflow/kernel.cpp | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 5f7f32358..981a8aabb 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -1451,7 +1451,7 @@ ral::execution::task_result WindowAggMergerKernel::do_process(std::vector< std:: try{ - output->addToCache(std::move(windowed_table)); + // output->addToCache(std::move(windowed_table)); }catch(const rmm::bad_alloc& e){ return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; }catch(const std::exception& e){ @@ -1478,8 +1478,7 @@ kstatus WindowAggMergerKernel::run() { ral::execution::executor::get_instance()->add_task( std::move(inputs), this->output_cache(), - this, - task_args); + this); cache_data = this->input_cache()->pullCacheData(); } diff --git a/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp b/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp index 16dad5a38..d12d8a596 100644 --- a/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp +++ b/engine/src/execution_graph/logic_controllers/taskflow/kernel.cpp @@ -36,7 +36,7 @@ std::shared_ptr kernel::output_cache(std::string cache std::shared_ptr kernel::input_cache(std::string cache_id) { cache_id = cache_id.empty() ? std::to_string(this->get_id()) : cache_id; - return this->input_.get_cache(kernel_id); + return this->input_.get_cache(cache_id); } bool kernel::add_to_output_cache(std::unique_ptr table, std::string cache_id, bool always_add) { From 454a79c81e5ffcf49c53eb013ea99d65bf796b19 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Fri, 23 Apr 2021 11:22:58 -0500 Subject: [PATCH 14/18] WIP --- .../BatchWindowFunctionProcessing.cpp | 67 ++++++++++++++++++- .../BatchWindowFunctionProcessing.h | 6 +- 2 files changed, 65 insertions(+), 8 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 981a8aabb..66f973b41 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -25,6 +25,10 @@ #include #include + +// WSM this could be a problematic dependency +#include + namespace ral { namespace batch { @@ -1450,8 +1454,9 @@ ral::execution::task_result WindowAggMergerKernel::do_process(std::vector< std:: std::unique_ptr & input = inputs[0]; try{ - - // output->addToCache(std::move(windowed_table)); + std::unique_ptr merged_aggs = std::make_unique( + ral::processor::evaluate_expressions(input->view(), merge_expressions), input->names()); + output->addToCache(std::move(merged_aggs)); }catch(const rmm::bad_alloc& e){ return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; }catch(const std::exception& e){ @@ -1461,14 +1466,70 @@ ral::execution::task_result WindowAggMergerKernel::do_process(std::vector< std:: return {ral::execution::task_status::SUCCESS, std::string(), std::vector< std::unique_ptr > ()}; } + +// WSM TODO: make this a task +std::vector> WindowAggMergerKernel::get_cumulative_aggregations_as_strings(){ + const std::string DELIM = "{|]~"; + std::unique_ptr cumulative_aggregations_table = this->input_cache("cumulative_aggregations")->pullFromCache(); + auto cumulative_aggregations_view = cumulative_aggregations_table->view(); + for (auto col_idx = 0; col_idx < cumulative_aggregations_table->num_columns(); col_idx++){ + std::string col_string = cudf::test::to_string(cumulative_aggregations_view.column(col_idx), DELIM); + cumulative_aggregations_str.push_back(StringUtil::split(col_string, DELIM) ); + } +} + +std::vector> WindowAggMergerKernel::get_merge_expressions_per_batch( + const std::vector> & cumulative_aggregations_str){ + + std::vector> merge_expressions_per_batch(this->aggs_wind_func.size()); + for (auto col_idx = 0; col_idx < this->aggs_wind_func.size(); col_idx++){ + merge_expressions_per_batch[col_idx].resize(cumulative_aggregations_str[col_idx].size()); + auto agg_func : this->aggs_wind_func[col_idx]; + for (auto row_idx = 0; cumulative_aggregations_str[col_idx].size(); row_idx++){ + if (cumulative_aggregations_str[col_idx][row_idx] == "null"){ + // if the cumulative_aggregation is null, we will just take the window function value (nothing to merge) + merge_expressions_per_batch[col_idx][row_idx] = "$" + std::to_string(col_idx); + } else { + if(agg_func == AggregateKind::SUM || agg_func == AggregateKind::SUM0 || + agg_func == AggregateKind::COUNT_VALID || agg_func == AggregateKind::COUNT_ALL || + agg_func == AggregateKind::ROW_NUMBER || agg_func == AggregateKind::COUNT_DISTINCT){ + // +($0, agg) + merge_expressions_per_batch[col_idx][row_idx] = + "+($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; + }else if(agg_func == AggregateKind::MIN){ + // CASE(<($0, agg),$0,agg) + merge_expressions_per_batch[col_idx][row_idx] = + "CASE(<($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + "),$" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; + + }else if(agg_func == AggregateKind::MAX){ + // CASE(>($0, agg),$0,agg) + merge_expressions_per_batch[col_idx][row_idx] = + "CASE(>($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + "),$" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; + + } + } + + }else if(agg_func == AggregateKind::NTH_ELEMENT && this->agg_param_values[col_idx] == 0){ //FIRST_VALUE + // WSM TODO + }else if(agg_func == AggregateKind::NTH_ELEMENT && this->agg_param_values[col_idx] == -1){ //LAST_VALUE + // WSM TODO + } else { + // WSM TODO ERROR + } + } +} + kstatus WindowAggMergerKernel::run() { CodeTimer timer; int self_node_idx = context->getNodeIndex(ral::communication::CommunicationData::getInstance().getSelfNode()); int num_nodes = context->getTotalNodes(); + std::vector> cumulative_aggregations_str = get_cumulative_aggregations_as_strings(); - cumulative_aggregations = this->input_cache("cumulative_aggregations")->pullFromCache(); + RAL_EXPECTS(cumulative_aggregations_str.size() == this->aggs_wind_func.size(), + "In WindowAggMergerKernel the number of aggregation window functions did not match the number of cumulative aggregation values received"); + std::unique_ptr cache_data = this->input_cache("batches")->pullCacheData(); while (cache_data != nullptr ){ diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index 2b7d9ff0e..352a02892 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -273,11 +273,7 @@ class WindowAggMergerKernel : public kernel { std::vector type_aggs_as_str; // ["MIN", "MAX"] std::vector aggs_wind_func; // [AggregateKind::MIN, AggregateKind::MAX] - - // WSM TODO. Do we really want this to be a BlazingTable member? - // It should always be pretty small, so it should not be a memory burden of any note, but it goes against the pattern - // We can always figure out a host representation that we would be more comfortable being left as a member - std::unique_ptr cumulative_aggregations; + std::vector> merge_expressions_per_batch; }; From 3703366fbd30f9ddda6c45f45a061cde438e70a5 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Fri, 23 Apr 2021 17:35:40 -0500 Subject: [PATCH 15/18] more WIP --- .../BatchWindowFunctionProcessing.cpp | 150 ++++++++++++------ .../BatchWindowFunctionProcessing.h | 9 ++ 2 files changed, 110 insertions(+), 49 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 66f973b41..2e301f22c 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -4,6 +4,7 @@ #include "execution_graph/logic_controllers/BlazingColumn.h" #include "execution_graph/logic_controllers/CacheData.h" +#include "execution_graph/logic_controllers/LogicalProject.h" #include "taskflow/executor.h" #include "CodeTimer.h" @@ -1446,66 +1447,99 @@ ral::execution::task_result WindowAggMergerKernel::do_process(std::vector< std:: std::shared_ptr output, cudaStream_t /*stream*/, const std::map& args) { - if (inputs.size() == 0) { return {ral::execution::task_status::SUCCESS, std::string(), std::vector< std::unique_ptr > ()}; } - std::unique_ptr & input = inputs[0]; + + auto& operation_type = args.at(TASK_ARG_OP_TYPE); + if (operation_type == MERGE_EXPRESSIONS) { - try{ - std::unique_ptr merged_aggs = std::make_unique( - ral::processor::evaluate_expressions(input->view(), merge_expressions), input->names()); - output->addToCache(std::move(merged_aggs)); - }catch(const rmm::bad_alloc& e){ - return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; - }catch(const std::exception& e){ - return {ral::execution::task_status::FAIL, std::string(e.what()), std::vector< std::unique_ptr > ()}; + try{ + std::vector> cumulative_aggregations_str = get_cumulative_aggregations_as_strings(input->view()); + + RAL_EXPECTS(cumulative_aggregations_str.size() == this->aggs_wind_func.size(), + "In WindowAggMergerKernel the number of aggregation window functions did not match the number of cumulative aggregation values received"); + + this->merge_expressions_per_batch = get_merge_expressions_per_batch(cumulative_aggregations_str); + }catch(const rmm::bad_alloc& e){ + return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; + }catch(const std::exception& e){ + return {ral::execution::task_status::FAIL, std::string(e.what()), std::vector< std::unique_ptr > ()}; + } + + } else if (operation_type == MERGE_AGGREGATION) { + auto batch_index_string = args.at(TASK_ARG_BATCH_INDEX); + int batch_index = std::stoi(args.at(TASK_ARG_BATCH_INDEX)); + + std::vector merge_expressions = this->merge_expressions_per_batch[batch_index]; + + try{ + std::unique_ptr merged_aggs = std::make_unique( + ral::processor::evaluate_expressions(input->view(), merge_expressions), input->names()); + output->addToCache(std::move(merged_aggs)); + }catch(const rmm::bad_alloc& e){ + return {ral::execution::task_status::RETRY, std::string(e.what()), std::move(inputs)}; + }catch(const std::exception& e){ + return {ral::execution::task_status::FAIL, std::string(e.what()), std::vector< std::unique_ptr > ()}; + } + } else { + throw std::runtime_error("WindowAggMergerKernel::do_process received unexpected operation type"); } return {ral::execution::task_status::SUCCESS, std::string(), std::vector< std::unique_ptr > ()}; } -// WSM TODO: make this a task -std::vector> WindowAggMergerKernel::get_cumulative_aggregations_as_strings(){ +std::vector> WindowAggMergerKernel::get_cumulative_aggregations_as_strings( + cudf::table_view cumulative_aggregations_view){ + + std::vector> cumulative_aggregations_str; const std::string DELIM = "{|]~"; - std::unique_ptr cumulative_aggregations_table = this->input_cache("cumulative_aggregations")->pullFromCache(); - auto cumulative_aggregations_view = cumulative_aggregations_table->view(); - for (auto col_idx = 0; col_idx < cumulative_aggregations_table->num_columns(); col_idx++){ + for (auto col_idx = 0; col_idx < cumulative_aggregations_view.num_columns(); col_idx++){ std::string col_string = cudf::test::to_string(cumulative_aggregations_view.column(col_idx), DELIM); cumulative_aggregations_str.push_back(StringUtil::split(col_string, DELIM) ); - } + } + return cumulative_aggregations_str; } std::vector> WindowAggMergerKernel::get_merge_expressions_per_batch( const std::vector> & cumulative_aggregations_str){ - std::vector> merge_expressions_per_batch(this->aggs_wind_func.size()); + std::vector> merge_expressions_per_batch(cumulative_aggregations_str[0].size()); + for (auto row_idx = 0; merge_expressions_per_batch.size(); row_idx++){ + merge_expressions_per_batch[row_idx].resize(cumulative_aggregations_str.size()); + } + for (auto col_idx = 0; col_idx < this->aggs_wind_func.size(); col_idx++){ - merge_expressions_per_batch[col_idx].resize(cumulative_aggregations_str[col_idx].size()); - auto agg_func : this->aggs_wind_func[col_idx]; - for (auto row_idx = 0; cumulative_aggregations_str[col_idx].size(); row_idx++){ - if (cumulative_aggregations_str[col_idx][row_idx] == "null"){ - // if the cumulative_aggregation is null, we will just take the window function value (nothing to merge) - merge_expressions_per_batch[col_idx][row_idx] = "$" + std::to_string(col_idx); - } else { - if(agg_func == AggregateKind::SUM || agg_func == AggregateKind::SUM0 || + + auto agg_func = this->aggs_wind_func[col_idx]; + if(agg_func == AggregateKind::SUM || agg_func == AggregateKind::SUM0 || agg_func == AggregateKind::COUNT_VALID || agg_func == AggregateKind::COUNT_ALL || - agg_func == AggregateKind::ROW_NUMBER || agg_func == AggregateKind::COUNT_DISTINCT){ - // +($0, agg) - merge_expressions_per_batch[col_idx][row_idx] = - "+($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; - }else if(agg_func == AggregateKind::MIN){ - // CASE(<($0, agg),$0,agg) - merge_expressions_per_batch[col_idx][row_idx] = - "CASE(<($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + "),$" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; - - }else if(agg_func == AggregateKind::MAX){ - // CASE(>($0, agg),$0,agg) - merge_expressions_per_batch[col_idx][row_idx] = - "CASE(>($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + "),$" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; - + agg_func == AggregateKind::ROW_NUMBER || agg_func == AggregateKind::COUNT_DISTINCT || + agg_func == AggregateKind::MIN || agg_func == AggregateKind::MAX){ + for (auto row_idx = 0; cumulative_aggregations_str[col_idx].size(); row_idx++){ + if (cumulative_aggregations_str[col_idx][row_idx] == "null"){ + // if the cumulative_aggregation is null, we will just take the window function value (nothing to merge) + merge_expressions_per_batch[row_idx][col_idx] = "$" + std::to_string(col_idx); + } else { + if(agg_func == AggregateKind::SUM || agg_func == AggregateKind::SUM0 || + agg_func == AggregateKind::COUNT_VALID || agg_func == AggregateKind::COUNT_ALL || + agg_func == AggregateKind::ROW_NUMBER || agg_func == AggregateKind::COUNT_DISTINCT){ + // +($0, agg) + merge_expressions_per_batch[row_idx][col_idx] = + "+($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; + }else if(agg_func == AggregateKind::MIN){ + // CASE(<($0, agg),$0,agg) + merge_expressions_per_batch[row_idx][col_idx] = + "CASE(<($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + "),$" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; + + }else if(agg_func == AggregateKind::MAX){ + // CASE(>($0, agg),$0,agg) + merge_expressions_per_batch[row_idx][col_idx] = + "CASE(>($" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + "),$" + std::to_string(col_idx) + "," + cumulative_aggregations_str[col_idx][row_idx] + ")"; + + } } } @@ -1517,6 +1551,7 @@ std::vector> WindowAggMergerKernel::get_merge_expressio // WSM TODO ERROR } } + return merge_expressions_per_batch; } kstatus WindowAggMergerKernel::run() { @@ -1525,26 +1560,43 @@ kstatus WindowAggMergerKernel::run() { int self_node_idx = context->getNodeIndex(ral::communication::CommunicationData::getInstance().getSelfNode()); int num_nodes = context->getTotalNodes(); - std::vector> cumulative_aggregations_str = get_cumulative_aggregations_as_strings(); - - RAL_EXPECTS(cumulative_aggregations_str.size() == this->aggs_wind_func.size(), - "In WindowAggMergerKernel the number of aggregation window functions did not match the number of cumulative aggregation values received"); + std::unique_ptr cumulative_aggregations_data = this->input_cache("cumulative_aggregations")->pullCacheData(); + std::vector> inputs; + inputs.push_back(std::move(cumulative_aggregations_data)); + + ral::execution::executor::get_instance()->add_task( + std::move(inputs), + this->output_cache(), + this, + {{TASK_ARG_OP_TYPE, MERGE_EXPRESSIONS}}); + + std::unique_lock lock(kernel_mutex); + kernel_cv.wait(lock,[this]{ + return this->tasks.empty(); + }); + + size_t batch_index = 0; std::unique_ptr cache_data = this->input_cache("batches")->pullCacheData(); while (cache_data != nullptr ){ - std::vector > inputs; - inputs.push_back(std::move(cache_data)); + std::vector > batch_inputs; + batch_inputs.push_back(std::move(cache_data)); + + if (batch_index >= this->merge_expressions_per_batch.size()){ + throw std::runtime_error("ERROR in WindowAggMergerKernel: Received more batches of data than expected from the number of cumulative aggregations"); + } ral::execution::executor::get_instance()->add_task( - std::move(inputs), - this->output_cache(), - this); + std::move(batch_inputs), + this->output_cache(), + this, + {{TASK_ARG_OP_TYPE, MERGE_AGGREGATION},{TASK_ARG_BATCH_INDEX, std::to_string(batch_index)}}); cache_data = this->input_cache()->pullCacheData(); + batch_index++; } - std::unique_lock lock(kernel_mutex); kernel_cv.wait(lock,[this]{ return this->tasks.empty(); }); diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h index 352a02892..2bb16d50f 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.h @@ -68,6 +68,7 @@ const std::string TASK_ARG_OVERLAP_SIZE="overlap_size"; const std::string TASK_ARG_SOURCE_BATCH_INDEX="source_batch_index"; const std::string TASK_ARG_TARGET_BATCH_INDEX="target_batch_index"; const std::string TASK_ARG_TARGET_NODE_INDEX="target_node_index"; +const std::string TASK_ARG_BATCH_INDEX="batch_index"; const std::string PRECEDING_OVERLAP_TYPE="preceding"; const std::string FOLLOWING_OVERLAP_TYPE="following"; @@ -76,6 +77,8 @@ const std::string PRECEDING_REQUEST="preceding_request"; const std::string FOLLOWING_REQUEST="following_request"; const std::string PRECEDING_RESPONSE="preceding_response"; const std::string FOLLOWING_RESPONSE="following_response"; +const std::string MERGE_EXPRESSIONS="merge_expressions"; +const std::string MERGE_AGGREGATION="merge_aggregation"; /** @@ -262,6 +265,12 @@ class WindowAggMergerKernel : public kernel { kstatus run() override; private: + + std::vector> get_merge_expressions_per_batch( + const std::vector> & cumulative_aggregations_str); + + std::vector> get_cumulative_aggregations_as_strings(cudf::table_view cumulative_aggregations_view); + // WindowAggMergerKernel(min_keys=[MIN($0) OVER (ORDER BY $3 DESC)], lag_col=[MAX($0) OVER (PARTITION BY $1)], n_name=[$2]) std::vector column_indices_partitioned; // column indices to be partitioned: [1] std::vector column_indices_ordered; // column indices to be ordered: [3] From af7aac6d201da6b526a628fa3aad8c3c8021c064 Mon Sep 17 00:00:00 2001 From: William Malpica Date: Mon, 26 Apr 2021 21:16:37 -0500 Subject: [PATCH 16/18] refactored test --- .../window_overlap_kernel_test.cpp | 197 +++++++----------- 1 file changed, 73 insertions(+), 124 deletions(-) diff --git a/engine/tests/kernel_tests/window_overlap_kernel_test.cpp b/engine/tests/kernel_tests/window_overlap_kernel_test.cpp index d587f791e..ff3a44192 100644 --- a/engine/tests/kernel_tests/window_overlap_kernel_test.cpp +++ b/engine/tests/kernel_tests/window_overlap_kernel_test.cpp @@ -26,8 +26,12 @@ using ral::frame::BlazingTable; using ral::cache::kernel; using Context = blazingdb::manager::Context; +struct kernel_test_base : public ::testing::Test { + + std::shared_ptr input_message_cache; + std::shared_ptr output_message_cache; + std::shared_ptr graph; -struct WindowOverlapAccumulatorTest : public ::testing::Test { virtual void SetUp() override { BlazingRMMInitialize(); float host_memory_quota=0.75; //default value @@ -36,46 +40,36 @@ struct WindowOverlapAccumulatorTest : public ::testing::Test { 4000000, 10, false, {}); int executor_threads = 10; ral::execution::executor::init_executor(executor_threads, 0.8); + + graph = std::make_shared(); + input_message_cache = std::make_shared(nullptr, "messages_in", false); + output_message_cache = std::make_shared(nullptr, "messages_out", false, ral::cache::CACHE_LEVEL_CPU ); + graph->set_input_and_output_caches(input_message_cache, output_message_cache); } virtual void TearDown() override { ral::memory::empty_pools(); BlazingRMMFinalize(); } + + }; -struct WindowOverlapGeneratorTest : public ::testing::Test { - virtual void SetUp() override { - BlazingRMMInitialize(); - float host_memory_quota=0.75; //default value - blazing_host_memory_resource::getInstance().initialize(host_memory_quota); - ral::memory::set_allocation_pools(4000000, 10, - 4000000, 10, false, {}); - int executor_threads = 10; - ral::execution::executor::init_executor(executor_threads, 0.8); - } - virtual void TearDown() override { - ral::memory::empty_pools(); - BlazingRMMFinalize(); - } +struct WindowOverlapAccumulatorTest : public kernel_test_base { + }; -struct WindowOverlapTest : public ::testing::Test { - virtual void SetUp() override { - BlazingRMMInitialize(); - float host_memory_quota=0.75; //default value - blazing_host_memory_resource::getInstance().initialize(host_memory_quota); - ral::memory::set_allocation_pools(4000000, 10, - 4000000, 10, false, {}); - int executor_threads = 10; - ral::execution::executor::init_executor(executor_threads, 0.8); - } +struct WindowOverlapGeneratorTest : public kernel_test_base { + +}; - virtual void TearDown() override { - ral::memory::empty_pools(); - BlazingRMMFinalize(); - } +struct WindowOverlapTest : public kernel_test_base { + +}; + +struct UnboundedWindowNoPartition : public kernel_test_base { + }; // Just creates a Context @@ -92,32 +86,6 @@ std::shared_ptr make_context(int num_nodes) { return context; } -// Creates a OverlapAccumulatorKernel using a valid `project_plan` -std::tuple, std::shared_ptr, std::shared_ptr> - make_overlap_Accumulator_kernel(std::string project_plan, std::shared_ptr context) { - std::size_t kernel_id = 1; - std::shared_ptr graph = std::make_shared(); - std::shared_ptr input_cache = std::make_shared(nullptr, "messages_in", false); - std::shared_ptr output_cache = std::make_shared(nullptr, "messages_out", false, ral::cache::CACHE_LEVEL_CPU ); - graph->set_input_and_output_caches(input_cache, output_cache); - std::shared_ptr overlap_accumulator_kernel = std::make_shared(kernel_id, project_plan, context, graph); - - return std::make_tuple(overlap_accumulator_kernel, input_cache, output_cache); -} - -// Creates a OverlapGeneratorKernel using a valid `project_plan` -std::tuple, std::shared_ptr, std::shared_ptr> -make_overlap_Generator_kernel(std::string project_plan, std::shared_ptr context) { - std::size_t kernel_id = 1; - std::shared_ptr graph = std::make_shared(); - std::shared_ptr input_cache = std::make_shared(nullptr, "messages_in", false); - std::shared_ptr output_cache = std::make_shared(nullptr, "messages_out", false, ral::cache::CACHE_LEVEL_CPU ); - graph->set_input_and_output_caches(input_cache, output_cache); - std::shared_ptr overlap_generator_kernel = std::make_shared(kernel_id, project_plan, context, graph); - - return std::make_tuple(overlap_generator_kernel, input_cache, output_cache); -} - // Creates two CacheMachines and register them with the `project_kernel` std::tuple, std::shared_ptr, std::shared_ptr, std::shared_ptr> register_kernel_overlap_accumulator_with_cache_machines( std::shared_ptr overlap_accumulator_kernel, @@ -369,11 +337,9 @@ TEST_F(WindowOverlapAccumulatorTest, BasicSingleNode) { communicationData.initialize("0", "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_cache, output_cache; - std::tie(overlap_accumulator_kernel, input_cache, output_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -467,11 +433,9 @@ TEST_F(WindowOverlapAccumulatorTest, BasicMultiNode_FirstNode) { communicationData.initialize(std::to_string(self_node_index), "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_message_cache, output_message_cache; - std::tie(overlap_accumulator_kernel, input_message_cache, output_message_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -601,11 +565,9 @@ TEST_F(WindowOverlapAccumulatorTest, BasicMultiNode_LastNode) { communicationData.initialize(std::to_string(self_node_index), "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_message_cache, output_message_cache; - std::tie(overlap_accumulator_kernel, input_message_cache, output_message_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -738,11 +700,9 @@ TEST_F(WindowOverlapAccumulatorTest, BasicMultiNode_MiddleNode) { communicationData.initialize(std::to_string(self_node_index), "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_message_cache, output_message_cache; - std::tie(overlap_accumulator_kernel, input_message_cache, output_message_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -914,12 +874,10 @@ TEST_F(WindowOverlapAccumulatorTest, BigWindowMultiNode_FirstNode) { communicationData.initialize(std::to_string(self_node_index), "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_message_cache, output_message_cache; - std::tie(overlap_accumulator_kernel, input_message_cache, output_message_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + - " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + + " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -1049,12 +1007,10 @@ TEST_F(WindowOverlapAccumulatorTest, BigWindowMultiNode_LastNode) { communicationData.initialize(std::to_string(self_node_index), "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_message_cache, output_message_cache; - std::tie(overlap_accumulator_kernel, input_message_cache, output_message_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + - " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + + " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -1191,12 +1147,10 @@ TEST_F(WindowOverlapAccumulatorTest, BigWindowMultiNode_MiddleNode) { communicationData.initialize(std::to_string(self_node_index), "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_message_cache, output_message_cache; - std::tie(overlap_accumulator_kernel, input_message_cache, output_message_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + - " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + + " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -1368,12 +1322,10 @@ TEST_F(WindowOverlapAccumulatorTest, BigWindowSingleNode) { communicationData.initialize(std::to_string(self_node_index), "/tmp"); // overlap kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_message_cache, output_message_cache; - std::tie(overlap_accumulator_kernel, input_message_cache, output_message_cache) = make_overlap_Accumulator_kernel( - "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + - " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])", context); - + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN " + std::to_string(preceding_value) + + " PRECEDING AND " + std::to_string(following_value) + " FOLLOWING)])"; + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine; std::tie(batchesCacheMachine, precedingCacheMachine, followingCacheMachine, outputCacheMachine) = register_kernel_overlap_accumulator_with_cache_machines( @@ -1418,12 +1370,12 @@ TEST_F(WindowOverlapAccumulatorTest, BigWindowSingleNode) { } std::tuple>, std::vector>, std::vector>> -run_overlap_generator_kernel(const std::string& project_plan, std::shared_ptr context, std::vector>& inputCacheData){ +run_overlap_generator_kernel(const std::string& project_plan, std::shared_ptr context, std::vector>& inputCacheData, + std::shared_ptr graph){ // overlap Generator kernel - std::shared_ptr overlap_generator_kernel; - std::shared_ptr input_generator_cache, output_generator_cache; - std::tie(overlap_generator_kernel, input_generator_cache, output_generator_cache) = make_overlap_Generator_kernel(project_plan, context); + + std::shared_ptr overlap_generator_kernel = std::make_shared(1, project_plan, context, graph); // register cache machines with the kernel std::shared_ptr batchesCacheMachineGenerator, @@ -1464,14 +1416,12 @@ run_overlap_accumulator_kernel(const std::string& project_plan, std::shared_ptr context, std::vector>& batchesCacheData, std::vector>& batchesPrecedingCacheData, - std::vector>& batchesFollowingCacheData){ + std::vector>& batchesFollowingCacheData, + std::shared_ptr graph){ // overlap Accumulator kernel - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_accumulator_cache, output_accumulator_cache; - std::tie(overlap_accumulator_kernel, input_accumulator_cache, output_accumulator_cache) = make_overlap_Accumulator_kernel( - project_plan, context); - + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); + // register cache machines with the kernel std::shared_ptr batchesCacheMachineAccumulator, precedingCacheMachineAccumulator, @@ -1562,7 +1512,8 @@ TEST_F(WindowOverlapGeneratorTest, BasicSingleNode) { batches_following_pulled) = run_overlap_generator_kernel(project_plan, context, - inputCacheData); + inputCacheData, + graph); EXPECT_EQ(batches_preceding_pulled.size(), preceding_overlaps.size()); EXPECT_EQ(batches_pulled.size(), expected_batch_out.size()); @@ -1649,7 +1600,8 @@ TEST_F(WindowOverlapGeneratorTest, BigWindowSingleNode) { batches_following_pulled) = run_overlap_generator_kernel(project_plan, context, - inputCacheData); + inputCacheData, + graph); EXPECT_EQ(batches_preceding_pulled.size(), preceding_overlaps.size()); EXPECT_EQ(batches_pulled.size(), expected_batch_out.size()); @@ -1745,13 +1697,15 @@ TEST_F(WindowOverlapTest, BasicSingleNode) { batches_following_pulled) = run_overlap_generator_kernel(project_plan, context, - inputCacheData); + inputCacheData, + graph); std::vector> last_batches_pulled = run_overlap_accumulator_kernel(project_plan, context, batches_pulled, batches_preceding_pulled, - batches_following_pulled); + batches_following_pulled, + graph); EXPECT_EQ(last_batches_pulled.size(), expected_out.size()); @@ -1812,13 +1766,15 @@ TEST_F(WindowOverlapTest, BigWindowSingleNode) { batches_following_pulled) = run_overlap_generator_kernel(project_plan, context, - inputCacheData); + inputCacheData, + graph); std::vector> last_batches_pulled = run_overlap_accumulator_kernel(project_plan, context, batches_pulled, batches_preceding_pulled, - batches_following_pulled); + batches_following_pulled, + graph); EXPECT_EQ(last_batches_pulled.size(), expected_out.size()); @@ -1869,15 +1825,8 @@ TEST_F(WindowOverlapTest, BasicSingleNode2) { std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])"; - // overlap Generator kernel - std::shared_ptr overlap_generator_kernel; - std::shared_ptr input_generator_cache, output_generator_cache; - std::tie(overlap_generator_kernel, input_generator_cache, output_generator_cache) = make_overlap_Generator_kernel(project_plan, context); - - std::shared_ptr overlap_accumulator_kernel; - std::shared_ptr input_accumulator_cache, output_accumulator_cache; - std::tie(overlap_accumulator_kernel, input_accumulator_cache, output_accumulator_cache) = make_overlap_Accumulator_kernel(project_plan, context); - + std::shared_ptr overlap_generator_kernel = std::make_shared(1, project_plan, context, graph); + std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); std::shared_ptr batchesCacheMachine = std::make_shared(context, "batches"); std::shared_ptr precedingCacheMachine = std::make_shared(context, "preceding_overlaps"); From c8c4d8c1514784ef57a1c28b777d3aaada94b1da Mon Sep 17 00:00:00 2001 From: William Malpica Date: Tue, 27 Apr 2021 20:17:07 -0500 Subject: [PATCH 17/18] fixed a bug and added a new test --- .../BatchWindowFunctionProcessing.cpp | 8 + .../logic_controllers/taskflow/kernel.h | 3 - .../logic_controllers/taskflow/kpair.h | 2 - .../window_overlap_kernel_test.cpp | 183 +++++++++++++++++- 4 files changed, 185 insertions(+), 11 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index 2e301f22c..a653051e1 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -327,6 +327,10 @@ OverlapGeneratorKernel::OverlapGeneratorKernel(std::size_t kernel_id, const std: this->output_.add_port("batches", "preceding_overlaps", "following_overlaps"); std::tie(this->preceding_value, this->following_value) = get_bounds_from_window_expression(this->expression); + + // for OverlapGeneratorKernel, we will treat UNBOUNDED windows as having values of 0 + this->preceding_value = this->preceding_value < 0 ? 0 : this->preceding_value; + this->following_value = this->following_value < 0 ? 0 : this->following_value; auto& self_node = ral::communication::CommunicationData::getInstance().getSelfNode(); self_node_index = context->getNodeIndex(self_node); @@ -468,6 +472,10 @@ OverlapAccumulatorKernel::OverlapAccumulatorKernel(std::size_t kernel_id, const std::tie(this->preceding_value, this->following_value) = get_bounds_from_window_expression(this->expression); + // for OverlapAccumulatorKernel, we will treat UNBOUNDED windows as having values of 0 + this->preceding_value = this->preceding_value < 0 ? 0 : this->preceding_value; + this->following_value = this->following_value < 0 ? 0 : this->following_value; + ral::cache::cache_settings cache_machine_config; cache_machine_config.type = ral::cache::CacheType::SIMPLE; cache_machine_config.context = context->clone(); diff --git a/engine/src/execution_graph/logic_controllers/taskflow/kernel.h b/engine/src/execution_graph/logic_controllers/taskflow/kernel.h index 852129045..bdbb2cba2 100644 --- a/engine/src/execution_graph/logic_controllers/taskflow/kernel.h +++ b/engine/src/execution_graph/logic_controllers/taskflow/kernel.h @@ -26,7 +26,6 @@ namespace ral { namespace cache { class kernel; class graph; -using kernel_pair = std::pair; /** * @brief This interface represents a computation unit in the execution graph. @@ -72,8 +71,6 @@ class kernel { virtual kstatus run() = 0; - kernel_pair operator[](const std::string & portname) { return std::make_pair(this, portname); } - /** * @brief Returns the kernel identifier. * diff --git a/engine/src/execution_graph/logic_controllers/taskflow/kpair.h b/engine/src/execution_graph/logic_controllers/taskflow/kpair.h index 7b4a741bf..5ed599e71 100644 --- a/engine/src/execution_graph/logic_controllers/taskflow/kpair.h +++ b/engine/src/execution_graph/logic_controllers/taskflow/kpair.h @@ -27,8 +27,6 @@ struct cache_settings { bool is_array_access = false; // is it a cache designated for array access }; -using kernel_pair = std::pair; - /** @brief A temporary object to represent a pair of two kernels linked into the execution graph. */ diff --git a/engine/tests/kernel_tests/window_overlap_kernel_test.cpp b/engine/tests/kernel_tests/window_overlap_kernel_test.cpp index ff3a44192..a86d24d02 100644 --- a/engine/tests/kernel_tests/window_overlap_kernel_test.cpp +++ b/engine/tests/kernel_tests/window_overlap_kernel_test.cpp @@ -11,6 +11,9 @@ #include "cudf_test/type_lists.hpp" // cudf::test::NumericTypes #include +#include +#include + #include "execution_graph/Context.h" #include "execution_graph/logic_controllers/taskflow/kernel.h" #include "execution_graph/logic_controllers/taskflow/graph.h" @@ -1784,7 +1787,26 @@ TEST_F(WindowOverlapTest, BigWindowSingleNode) { } } -TEST_F(WindowOverlapTest, BasicSingleNode2) { + +class WindowOverlapTestParam : + public WindowOverlapTest, + public ::testing::WithParamInterface> { +}; + +INSTANTIATE_TEST_SUITE_P( + TimeoutTests, + WindowOverlapTestParam, + ::testing::Values( + std::make_tuple(50, 10, "ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING"), + std::make_tuple(0, 0, "ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"), + std::make_tuple(0, 0, ""), + std::make_tuple(0, 10, "ROWS BETWEEN UNBOUNDED PRECEDING AND 10 FOLLOWING"), + std::make_tuple(10, 0, "ROWS BETWEEN 10 PRECEDING AND UNBOUNDED FOLLOWING"), + std::make_tuple(0, 10, "ROWS BETWEEN CURRENT ROW AND 10 FOLLOWING"), + std::make_tuple(10, 0, "ROWS BETWEEN 10 PRECEDING AND CURRENT ROW") + )); + +TEST_P(WindowOverlapTestParam, BasicSingleNodeParam) { size_t size = 100000; @@ -1800,8 +1822,11 @@ TEST_F(WindowOverlapTest, BasicSingleNode2) { CudfTableView full_data_cudf_view ({col0, col1, col2}); - int preceding_value = 50; - int following_value = 10; + int preceding_value = std::get<0>(GetParam()); + int following_value = std::get<1>(GetParam()); + std::string frame_str = std::get<2>(GetParam()); + std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 " + frame_str + ")])"; + std::vector batch_sizes = {5000, 50000, 20000, 15000, 10000}; // need to sum up to size // define how its broken up into batches and overlaps @@ -1823,8 +1848,6 @@ TEST_F(WindowOverlapTest, BasicSingleNode2) { auto & communicationData = ral::communication::CommunicationData::getInstance(); communicationData.initialize("0", "/tmp"); - std::string project_plan = "LogicalProject(min_val=[MIN($0) OVER (ORDER BY $1 ROWS BETWEEN 50 PRECEDING AND 10 FOLLOWING)])"; - std::shared_ptr overlap_generator_kernel = std::make_shared(1, project_plan, context, graph); std::shared_ptr overlap_accumulator_kernel = std::make_shared(1, project_plan, context, graph); @@ -1887,4 +1910,152 @@ TEST_F(WindowOverlapTest, BasicSingleNode2) { auto table_out = last_batches_pulled[i]->decache(); cudf::test::expect_tables_equivalent(expected_out[i]->view(), table_out->view()); } -} \ No newline at end of file +} + + +std::unique_ptr make_expected_window_function_output(CudfTableView full_data_cudf_view, + std::vector names, + int preceding_value, + int following_value, + std::vector agg_col_indices, + std::vector> aggregations){ + + std::vector> agg_results; + for (int col_ind = 0; col_ind < agg_col_indices.size(); col_ind++){ + std::unique_ptr windowed_col = cudf::rolling_window(full_data_cudf_view.column(col_ind), + preceding_value + 1, following_value, 1, std::move(aggregations[col_ind])); + agg_results.push_back(std::move(windowed_col)); + } + std::unique_ptr result_table = std::make_unique(std::move(agg_results)); + return std::make_unique(std::move(result_table), names); +} + + + +TEST_F(UnboundedWindowNoPartition, BasicSingleNode) { + size_t size = 100000; + + // Define full dataset + auto iter0 = cudf::detail::make_counting_transform_iterator(0, [](auto i) { return int32_t(i);}); + auto iter1 = cudf::detail::make_counting_transform_iterator(1000, [](auto i) { return int32_t(i * 2);}); + auto iter2 = cudf::detail::make_counting_transform_iterator(1000000, [](auto i) { return int32_t(-i*3);}); + auto valids_iter = cudf::detail::make_counting_transform_iterator(0, [](auto /*i*/) { return true; }); + + cudf::test::fixed_width_column_wrapper col0(iter0, iter0 + size, valids_iter); + cudf::test::fixed_width_column_wrapper col1(iter1, iter1 + size, valids_iter); + cudf::test::fixed_width_column_wrapper col2(iter2, iter2 + size, valids_iter); + + CudfTableView full_data_cudf_view ({col0, col1, col2}); + + int preceding_value = 0; + int following_value = 0; + std::string frame_str = "ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"; + std::string project_plan = "LogicalProject(min_val1=[MIN($1) OVER (ORDER BY $0 " + frame_str + ")],"; + project_plan = project_plan + "max_val1=[MAX($1) OVER (ORDER BY $0 " + frame_str + ")],"; + project_plan = project_plan + "count_val1=[COUNT($1) OVER (ORDER BY $0 " + frame_str + ")],"; + project_plan = project_plan + "sum_val1=[SUM($1) OVER (ORDER BY $0 " + frame_str + ")],"; + project_plan = project_plan + "min_val2=[MIN($2) OVER (ORDER BY $0 " + frame_str + ")],"; + project_plan = project_plan + "max_val2=[MAX($2) OVER (ORDER BY $0 " + frame_str + ")],"; + project_plan = project_plan + "count_val2=[COUNT($2) OVER (ORDER BY $0 " + frame_str + ")],"; + project_plan = project_plan + "sum_val2=[SUM($2) OVER (ORDER BY $0 " + frame_str + ")])"; + + std::vector batch_sizes = {5000, 50000, 20000, 15000, 10000}; // need to sum up to size + + // define how its broken up into batches and overlaps + std::vector names({"A", "B", "C"}); + + std::vector> intput_batches = make_expected_accumulator_output(full_data_cudf_view, + preceding_value, + following_value, + batch_sizes, names); + + // make the expected output + std::vector agg_col_indices = {1, 1, 1, 1, 2, 2, 2, 2}; + std::vector> aggregations; + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::MIN, 0)); + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::MAX, 0)); + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::COUNT_VALID, 0)); + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::SUM, 0)); + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::MIN, 0)); + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::MAX, 0)); + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::COUNT_VALID, 0)); + aggregations.push_back(ral::operators::makeCudfAggregation(AggregateKind::SUM, 0)); + + std::unique_ptr full_expected_out = make_expected_window_function_output(full_data_cudf_view, + names, + preceding_value, + following_value, + agg_col_indices, + std::move(aggregations)); + std::vector split_indexes = batch_sizes; + std::partial_sum(split_indexes.begin(), split_indexes.end(), split_indexes.begin()); + split_indexes.erase(split_indexes.begin() + split_indexes.size() - 1); + auto expected_batch_views = cudf::split(full_expected_out->view(), split_indexes); + + + // create and start kernel + // Context + std::shared_ptr context = make_context(1); + + auto & communicationData = ral::communication::CommunicationData::getInstance(); + communicationData.initialize("0", "/tmp"); + + std::shared_ptr compute_window_unbounded_kernel = std::make_shared(1, project_plan, context, graph); + std::shared_ptr window_agg_merger_kernel = std::make_shared(1, project_plan, context, graph); + + std::shared_ptr inputCacheMachine = std::make_shared(context, "1"); + std::shared_ptr batchesCacheMachine = std::make_shared(context, "batches"); + std::shared_ptr cumulativeAggCacheMachine = std::make_shared(context, "cumulative_aggregations"); + std::shared_ptr outputCacheMachine = std::make_shared(context, "1"); + + compute_window_unbounded_kernel->input_.register_cache("1", inputCacheMachine); + compute_window_unbounded_kernel->output_.register_cache("batches", batchesCacheMachine); + compute_window_unbounded_kernel->output_.register_cache("cumulative_aggregations", cumulativeAggCacheMachine); + + window_agg_merger_kernel->input_.register_cache("batches", batchesCacheMachine); + window_agg_merger_kernel->input_.register_cache("cumulative_aggregations", cumulativeAggCacheMachine); + window_agg_merger_kernel->output_.register_cache("1", outputCacheMachine); + + + // run function in compute_window_unbounded_kernel + std::thread run_thread_1 = std::thread([compute_window_unbounded_kernel]() { + kstatus process = compute_window_unbounded_kernel->run(); + EXPECT_EQ(kstatus::proceed, process); + }); + + // run function in window_agg_merger_kernel + std::thread run_thread_2 = std::thread([window_agg_merger_kernel](){ + kstatus process = window_agg_merger_kernel->run(); + EXPECT_EQ(kstatus::proceed, process); + }); + + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + + // add data into the input CacheMachines + for (std::size_t i = 0; i < intput_batches.size(); i++) { + inputCacheMachine->addToCache(std::move(intput_batches[i])); + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + } + inputCacheMachine->finish(); + + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + + batchesCacheMachine->finish(); + cumulativeAggCacheMachine->finish(); + + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + outputCacheMachine->finish(); + + run_thread_1.join(); + run_thread_2.join(); + + // get and validate output + auto last_batches_pulled = outputCacheMachine->pull_all_cache_data(); + + EXPECT_EQ(last_batches_pulled.size(), expected_batch_views.size()); + + for (std::size_t i = 0; i < last_batches_pulled.size(); i++) { + auto table_out = last_batches_pulled[i]->decache(); + cudf::test::expect_tables_equivalent(expected_batch_views[i], table_out->view()); + } +} From f7921f999668f4709954be4dc91d2d886c22e04f Mon Sep 17 00:00:00 2001 From: William Malpica Date: Wed, 28 Apr 2021 10:30:49 -0500 Subject: [PATCH 18/18] WIP --- .../BatchWindowFunctionProcessing.cpp | 11 +++++------ .../kernel_tests/window_overlap_kernel_test.cpp | 14 ++++++++------ 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp index a653051e1..68bd39bb9 100644 --- a/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp +++ b/engine/src/execution_graph/logic_controllers/BatchWindowFunctionProcessing.cpp @@ -1134,12 +1134,11 @@ std::unique_ptr ComputeWindowKernelUnbounded::compute_column_from_wi windowed_col = cudf::grouped_rolling_window(partitioned_table_view, col_view_to_agg, col_view_to_agg.size(), col_view_to_agg.size(), 1, window_aggregation); } } else { - if (window_expression_contains_bounds(this->expression)) { - // TODO: for now just ROWS bounds works (not RANGE) - windowed_col = cudf::rolling_window(col_view_to_agg, this->preceding_value + 1, this->following_value, 1, window_aggregation); - } else { - throw std::runtime_error("Window functions without partitions and without bounded windows are currently not supported"); - } + // TODO: for now just ROWS bounds works (not RANGE) + windowed_col = cudf::rolling_window(col_view_to_agg, + this->preceding_value >= 0 ? this->preceding_value + 1: col_view_to_agg.size(), + this->following_value >= 0 ? this->following_value : col_view_to_agg.size(), + 1, window_aggregation); } return std::move(windowed_col); diff --git a/engine/tests/kernel_tests/window_overlap_kernel_test.cpp b/engine/tests/kernel_tests/window_overlap_kernel_test.cpp index a86d24d02..4953d3fc6 100644 --- a/engine/tests/kernel_tests/window_overlap_kernel_test.cpp +++ b/engine/tests/kernel_tests/window_overlap_kernel_test.cpp @@ -41,7 +41,7 @@ struct kernel_test_base : public ::testing::Test { blazing_host_memory_resource::getInstance().initialize(host_memory_quota); ral::memory::set_allocation_pools(4000000, 10, 4000000, 10, false, {}); - int executor_threads = 10; + int executor_threads = 1; // WMS TODO reset this back to 10 ral::execution::executor::init_executor(executor_threads, 0.8); graph = std::make_shared(); @@ -1922,8 +1922,10 @@ std::unique_ptr make_expected_window_function_output(CudfTableView std::vector> agg_results; for (int col_ind = 0; col_ind < agg_col_indices.size(); col_ind++){ - std::unique_ptr windowed_col = cudf::rolling_window(full_data_cudf_view.column(col_ind), - preceding_value + 1, following_value, 1, std::move(aggregations[col_ind])); + std::unique_ptr windowed_col = cudf::rolling_window(full_data_cudf_view.column(agg_col_indices[col_ind]), + preceding_value >= 0 ? preceding_value + 1: full_data_cudf_view.num_rows(), + following_value >= 0 ? following_value : full_data_cudf_view.num_rows(), + 1, std::move(aggregations[col_ind])); agg_results.push_back(std::move(windowed_col)); } std::unique_ptr result_table = std::make_unique(std::move(agg_results)); @@ -1947,7 +1949,7 @@ TEST_F(UnboundedWindowNoPartition, BasicSingleNode) { CudfTableView full_data_cudf_view ({col0, col1, col2}); - int preceding_value = 0; + int preceding_value = -1; int following_value = 0; std::string frame_str = "ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"; std::string project_plan = "LogicalProject(min_val1=[MIN($1) OVER (ORDER BY $0 " + frame_str + ")],"; @@ -1965,8 +1967,8 @@ TEST_F(UnboundedWindowNoPartition, BasicSingleNode) { std::vector names({"A", "B", "C"}); std::vector> intput_batches = make_expected_accumulator_output(full_data_cudf_view, - preceding_value, - following_value, + preceding_value < 0 ? 0 : preceding_value, + following_value < 0 ? 0 : following_value, batch_sizes, names); // make the expected output