From 46eefa1617ca5e43fa03c581a03a78e1e76d6ee5 Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Thu, 12 Aug 2021 11:01:40 -0700 Subject: [PATCH 01/51] Add dask-sql query files --- .../queries/q01/gpu_bdb_query_01_dask_sql.py | 107 +++++ .../queries/q02/gpu_bdb_query_02_dask_sql.py | 112 ++++++ .../queries/q03/gpu_bdb_query_03_dask_sql.py | 215 ++++++++++ .../queries/q04/gpu_bdb_query_04_dask_sql.py | 182 +++++++++ .../queries/q05/gpu_bdb_query_05_dask_sql.py | 191 +++++++++ .../queries/q06/gpu_bdb_query_06_dask_sql.py | 163 ++++++++ .../queries/q07/gpu_bdb_query_07_dask_sql.py | 118 ++++++ .../queries/q08/gpu_bdb_query_08_dask_sql.py | 281 +++++++++++++ .../queries/q09/gpu_bdb_query_09_dask_sql.py | 183 +++++++++ .../queries/q10/gpu_bdb_query_10_dask_sql.py | 170 ++++++++ .../queries/q11/gpu_bdb_query_11_dask_sql.py | 108 +++++ .../queries/q12/gpu_bdb_query_12_dask_sql.py | 101 +++++ .../queries/q13/gpu_bdb_query_13_dask_sql.py | 139 +++++++ .../queries/q14/gpu_bdb_query_14_dask_sql.py | 97 +++++ .../queries/q15/gpu_bdb_query_15_dask_sql.py | 102 +++++ .../queries/q16/gpu_bdb_query_16_dask_sql.py | 138 +++++++ .../queries/q17/gpu_bdb_query_17_dask_sql.py | 141 +++++++ .../queries/q18/gpu_bdb_query_18_dask_sql.py | 360 +++++++++++++++++ .../queries/q19/gpu_bdb_query_19_dask_sql.py | 212 ++++++++++ .../queries/q20/gpu_bdb_query_20_dask_sql.py | 168 ++++++++ .../queries/q21/gpu_bdb_query_21_dask_sql.py | 173 ++++++++ .../queries/q22/gpu_bdb_query_22_dask_sql.py | 123 ++++++ .../queries/q23/gpu_bdb_query_23_dask_sql.py | 160 ++++++++ .../queries/q24/gpu_bdb_query_24_dask_sql.py | 123 ++++++ .../queries/q25/gpu_bdb_query_25_dask_sql.py | 153 +++++++ .../queries/q26/gpu_bdb_query_26_dask_sql.py | 131 ++++++ .../queries/q27/gpu_bdb_query_27_dask_sql.py | 152 +++++++ .../queries/q28/gpu_bdb_query_28_dask_sql.py | 372 ++++++++++++++++++ .../queries/q29/gpu_bdb_query_29_dask_sql.py | 93 +++++ .../queries/q30/gpu_bdb_query_30_dask_sql.py | 131 ++++++ 30 files changed, 4899 insertions(+) create mode 100755 gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py create mode 100755 gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py create mode 100755 gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py create mode 100755 gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py create mode 100755 gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py create mode 100755 gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py create mode 100755 gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py create mode 100755 gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py create mode 100755 gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py create mode 100755 gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py create mode 100755 gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py create mode 100755 gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py create mode 100644 gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py create mode 100755 gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py create mode 100755 gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py create mode 100755 gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py create mode 100755 gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py create mode 100755 gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py create mode 100755 gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py create mode 100755 gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py create mode 100755 gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py create mode 100755 gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py create mode 100755 gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py create mode 100755 gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py create mode 100755 gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py create mode 100755 gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py create mode 100755 gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py create mode 100755 gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py create mode 100755 gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py create mode 100755 gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py new file mode 100755 index 00000000..11ff194d --- /dev/null +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -0,0 +1,107 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +import os + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + + +# -------- Q1 ----------- +q01_i_category_id_IN = "1, 2, 3" +# -- sf1 -> 11 stores, 90k sales in 820k lines +q01_ss_store_sk_IN = "10, 20, 33, 40, 50" +q01_viewed_together_count = 50 +q01_limit = 100 + + +item_cols = ["i_item_sk", "i_category_id"] +ss_cols = ["ss_item_sk", "ss_store_sk", "ss_ticket_number"] + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_df = table_reader.read("item", relevant_cols=item_cols) + ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) + + bc.create_table("item", item_df) + bc.create_table("store_sales", ss_df) + + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_distinct = f""" + SELECT DISTINCT ss_item_sk, ss_ticket_number + FROM store_sales s, item i + WHERE s.ss_item_sk = i.i_item_sk + AND i.i_category_id IN ({q01_i_category_id_IN}) + AND s.ss_store_sk IN ({q01_ss_store_sk_IN}) + """ + result_distinct = bc.sql(query_distinct) + + result_distinct = result_distinct.persist() + wait(result_distinct) + bc.create_table("distinct_table", result_distinct) + + query = f""" + SELECT item_sk_1, item_sk_2, COUNT(*) AS cnt + FROM + ( + SELECT CAST(t1.ss_item_sk as BIGINT) AS item_sk_1, + CAST(t2.ss_item_sk AS BIGINT) AS item_sk_2 + FROM distinct_table t1 + INNER JOIN distinct_table t2 + ON t1.ss_ticket_number = t2.ss_ticket_number + WHERE t1.ss_item_sk < t2.ss_item_sk + ) + GROUP BY item_sk_1, item_sk_2 + HAVING COUNT(*) > {q01_viewed_together_count} + ORDER BY cnt DESC, CAST(item_sk_1 AS VARCHAR), + CAST(item_sk_2 AS VARCHAR) + LIMIT {q01_limit} + """ + result = bc.sql(query) + + bc.drop_table("distinct_table") + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, bc = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py new file mode 100755 index 00000000..149b660e --- /dev/null +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -0,0 +1,112 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader +from bdb_tools.sessionization import get_distinct_sessions +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q2 ----------- +q02_item_sk = 10001 +q02_limit = 30 +q02_session_timeout_inSec = 3600 + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + bc.create_table("web_clickstreams", wcs_df) + # bc.create_table("web_clickstreams", + # os.path.join(data_dir, "web_clickstreams/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = """ + SELECT + CAST(wcs_user_sk AS INTEGER) AS wcs_user_sk, + CAST(wcs_item_sk AS INTEGER) AS wcs_item_sk, + (wcs_click_date_sk * 86400 + wcs_click_time_sk) AS tstamp_inSec + FROM web_clickstreams + WHERE wcs_item_sk IS NOT NULL + AND wcs_user_sk IS NOT NULL + ORDER BY wcs_user_sk + """ + wcs_result = bc.sql(query_1) + + session_df = wcs_result.map_partitions( + get_distinct_sessions, + keep_cols=["wcs_user_sk", "wcs_item_sk"], + time_out=q02_session_timeout_inSec, + ) + del wcs_result + + session_df = session_df.persist() + wait(session_df) + bc.create_table('session_df', session_df) + + last_query = f""" + WITH item_df AS ( + SELECT wcs_user_sk, session_id + FROM session_df + WHERE wcs_item_sk = {q02_item_sk} + ) + SELECT sd.wcs_item_sk as item_sk_1, + count(sd.wcs_item_sk) as cnt + FROM session_df sd + INNER JOIN item_df id + ON sd.wcs_user_sk = id.wcs_user_sk + AND sd.session_id = id.session_id + AND sd.wcs_item_sk <> {q02_item_sk} + GROUP BY sd.wcs_item_sk + ORDER BY cnt desc + LIMIT {q02_limit} + """ + result = bc.sql(last_query) + result["item_sk_2"] = q02_item_sk + result_order = ["item_sk_1", "item_sk_2", "cnt"] + result = result[result_order] + + del session_df + bc.drop_table("session_df") + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py new file mode 100755 index 00000000..7b89eac1 --- /dev/null +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -0,0 +1,215 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster +from numba import cuda + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q03 ----------- +q03_days_in_sec_before_purchase = 864000 +q03_views_before_purchase = 5 +q03_purchased_item_IN = 10001 +# --see q1 for categories +q03_purchased_item_category_IN = "2,3" +q03_limit = 100 + + +@cuda.jit +def find_items_viewed_before_purchase_kernel( + relevant_idx_col, user_col, timestamp_col, item_col, out_col, N +): + """ + Find the past N items viewed after a relevant purchase was made, + as defined by the configuration of this query. + """ + i = cuda.grid(1) + relevant_item = q03_purchased_item_IN + + if i < (relevant_idx_col.size): # boundary guard + # every relevant row gets N rows in the output, so we need to map the indexes + # back into their position in the original array + orig_idx = relevant_idx_col[i] + current_user = user_col[orig_idx] + + # look at the previous N clicks (assume sorted descending) + rows_to_check = N + remaining_rows = user_col.size - orig_idx + + if remaining_rows <= rows_to_check: + rows_to_check = remaining_rows - 1 + + for k in range(1, rows_to_check + 1): + if current_user != user_col[orig_idx + k]: + out_col[i * N + k - 1] = 0 + + # only checking relevant purchases via the relevant_idx_col + elif (timestamp_col[orig_idx + k] <= timestamp_col[orig_idx]) & ( + timestamp_col[orig_idx + k] + >= (timestamp_col[orig_idx] - q03_days_in_sec_before_purchase) + ): + out_col[i * N + k - 1] = item_col[orig_idx + k] + else: + out_col[i * N + k - 1] = 0 + + +def apply_find_items_viewed(df, item_mappings): + import cudf + + # need to sort descending to ensure that the + # next N rows are the previous N clicks + df = df.sort_values( + by=["wcs_user_sk", "tstamp", "wcs_sales_sk", "wcs_item_sk"], + ascending=[False, False, False, False], + ) + df.reset_index(drop=True, inplace=True) + df["relevant_flag"] = (df.wcs_sales_sk != 0) & ( + df.wcs_item_sk == q03_purchased_item_IN + ) + df["relevant_idx_pos"] = df.index.to_series() + df.reset_index(drop=True, inplace=True) + # only allocate output for the relevant rows + sample = df.loc[df.relevant_flag == True] + sample.reset_index(drop=True, inplace=True) + + N = q03_views_before_purchase + size = len(sample) + + # we know this can be int32, since it's going to contain item_sks + out_arr = cuda.device_array(size * N, dtype=df["wcs_item_sk"].dtype) + + find_items_viewed_before_purchase_kernel.forall(size)( + sample["relevant_idx_pos"], + df["wcs_user_sk"], + df["tstamp"], + df["wcs_item_sk"], + out_arr, + N, + ) + + result = cudf.DataFrame({"prior_item_viewed": out_arr}) + + del out_arr + del df + del sample + + filtered = result.merge( + item_mappings, + how="inner", + left_on=["prior_item_viewed"], + right_on=["i_item_sk"], + ) + return filtered + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_cols = ["i_category_id", "i_item_sk"] + item_df = table_reader.read("item", relevant_cols=item_cols) + wcs_df = table_reader.read("web_clickstreams") + + bc.create_table("web_clickstreams", wcs_df) + bc.create_table("item", item_df) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = """ + SELECT i_item_sk, + CAST(i_category_id AS TINYINT) AS i_category_id + FROM item + """ + item_df = bc.sql(query_1) + + item_df = item_df.persist() + wait(item_df) + bc.create_table("item_df", item_df) + + query_2 = """ + SELECT CAST(w.wcs_user_sk AS INTEGER) as wcs_user_sk, + wcs_click_date_sk * 86400 + wcs_click_time_sk AS tstamp, + CAST(w.wcs_item_sk AS INTEGER) as wcs_item_sk, + CAST(COALESCE(w.wcs_sales_sk, 0) AS INTEGER) as wcs_sales_sk + FROM web_clickstreams AS w + INNER JOIN item_df AS i ON w.wcs_item_sk = i.i_item_sk + WHERE w.wcs_user_sk IS NOT NULL + AND w.wcs_item_sk IS NOT NULL + ORDER BY w.wcs_user_sk + """ + merged_df = bc.sql(query_2) + + query_3 = f""" + SELECT i_item_sk, i_category_id + FROM item_df + WHERE i_category_id IN ({q03_purchased_item_category_IN}) + """ + item_df_filtered = bc.sql(query_3) + + product_view_results = merged_df.map_partitions( + apply_find_items_viewed, item_mappings=item_df_filtered + ) + + product_view_results = product_view_results.persist() + wait(product_view_results) + + bc.drop_table("item_df") + del item_df + del merged_df + del item_df_filtered + + bc.create_table('product_result', product_view_results) + + last_query = f""" + SELECT CAST({q03_purchased_item_IN} AS BIGINT) AS purchased_item, + i_item_sk AS lastviewed_item, + COUNT(i_item_sk) AS cnt + FROM product_result + GROUP BY i_item_sk + ORDER BY purchased_item, cnt desc, lastviewed_item + LIMIT {q03_limit} + """ + result = bc.sql(last_query) + + bc.drop_table("product_result") + del product_view_results + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py new file mode 100755 index 00000000..725cd5da --- /dev/null +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -0,0 +1,182 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster +from bdb_tools.sessionization import get_sessions + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +def abandonedShoppingCarts(df, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): + import cudf + # work around for https://github.com/rapidsai/cudf/issues/5470 + df.reset_index(drop=True, inplace=True) + + # Select groups where last dynamic row comes after last order row + filtered_df = df[ + (df["wp_type_codes"] == ORDER_CAT_CODE) + | (df["wp_type_codes"] == DYNAMIC_CAT_CODE) + ] + # work around for https://github.com/rapidsai/cudf/issues/5470 + filtered_df.reset_index(drop=True, inplace=True) + # Create a new column that is the concatenation of timestamp and wp_type_codes + # (eg:123456:3, 234567:5) + filtered_df["wp_type_codes"] = ( + filtered_df["tstamp_inSec"] + .astype("str") + .str.cat(filtered_df["wp_type_codes"].astype("str"), sep=":") + ) + # This gives the last occurrence (by timestamp) within the "order", "dynamic" wp_types + filtered_df = filtered_df.groupby( + ["wcs_user_sk", "session_id"], as_index=False, sort=False + ).agg({"wp_type_codes": "max"}) + # If the max contains dynamic, keep the row else discard. + last_dynamic_df = filtered_df[ + filtered_df["wp_type_codes"].str.contains( + ":" + str(DYNAMIC_CAT_CODE), regex=False + ) + ] + del filtered_df + + # Find counts for each group + grouped_count_df = df.groupby( + ["wcs_user_sk", "session_id"], as_index=False, sort=False + ).agg({"tstamp_inSec": "count"}) + # Merge counts with the "dynamic" shopping cart groups + result = last_dynamic_df.merge( + grouped_count_df, on=["wcs_user_sk", "session_id"], how="inner" + ) + del (last_dynamic_df, grouped_count_df) + return cudf.DataFrame( + {"pagecount": result.tstamp_inSec.sum(), "count": len(result)} + ) + + +def reduction_function(df, keep_cols, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): + df = get_sessions(df, keep_cols=keep_cols) + df = abandonedShoppingCarts( + df, DYNAMIC_CAT_CODE=DYNAMIC_CAT_CODE, ORDER_CAT_CODE=ORDER_CAT_CODE + ) + return df + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + wp_cols = ["wp_type", "wp_web_page_sk"] + wp_df = table_reader.read("web_page", relevant_cols=wp_cols) + + wcs_cols = [ + "wcs_user_sk", + "wcs_click_date_sk", + "wcs_click_time_sk", + "wcs_web_page_sk", + "wcs_sales_sk", + ] + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + bc.create_table('web_page_wo_categorical', wp_df) + bc.create_table('web_clickstreams', wcs_df) + + # bc.create_table('web_page_wo_categorical', os.path.join(data_dir, "web_page/*.parquet")) + # bc.create_table('web_clickstreams', + # os.path.join(data_dir, "web_clickstreams/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_web_page = """ + SELECT wp_type, wp_web_page_sk + FROM web_page_wo_categorical + """ + wp = bc.sql(query_web_page) + + # Convert wp_type to categorical and get cat_id of review and dynamic type + wp["wp_type"] = wp["wp_type"].map_partitions( + lambda ser: ser.astype("category")) + + cpu_categories = wp["wp_type"].compute().cat.categories + if hasattr(cpu_categories, "to_pandas"): + cpu_categories = cpu_categories.to_pandas() + + DYNAMIC_CAT_CODE = cpu_categories.get_loc("dynamic") + ORDER_CAT_CODE = cpu_categories.get_loc("order") + + # ### cast to minimum viable dtype + import cudf + codes_min_signed_type = cudf.utils.dtypes.min_signed_type( + len(cpu_categories)) + wp["wp_type_codes"] = wp["wp_type"].cat.codes.astype(codes_min_signed_type) + wp["wp_type"] = wp["wp_type"].cat.codes.astype(codes_min_signed_type) + cols_2_keep = ["wp_web_page_sk", "wp_type_codes"] + wp = wp[cols_2_keep] + + wp = wp.persist() + wait(wp) + bc.create_table('web_page', wp) + + query = """ + SELECT + c.wcs_user_sk, + w.wp_type_codes, + (wcs_click_date_sk * 86400 + wcs_click_time_sk) AS tstamp_inSec + FROM web_clickstreams c, web_page w + WHERE c.wcs_web_page_sk = w.wp_web_page_sk + AND c.wcs_web_page_sk IS NOT NULL + AND c.wcs_user_sk IS NOT NULL + AND c.wcs_sales_sk IS NULL --abandoned implies: no sale + ORDER BY wcs_user_sk, tstamp_inSec + """ + merged_df = bc.sql(query) + + keep_cols = ["wcs_user_sk", "wp_type_codes", "tstamp_inSec"] + result_df = merged_df.map_partitions( + reduction_function, keep_cols, DYNAMIC_CAT_CODE, ORDER_CAT_CODE + ) + + result = result_df["pagecount"].sum() / result_df["count"].sum() + # Persist before computing to ensure scalar transfer only on compute + result = result.persist() + + result = result.compute() + result_df = cudf.DataFrame({"sum(pagecount)/count(*)": [result]}) + bc.drop_table("web_page") + return result_df + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py new file mode 100755 index 00000000..af90eded --- /dev/null +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -0,0 +1,191 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +from dask_cuda import LocalCUDACluster +from dask.distributed import Client, wait +from dask import delayed +import os + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) +from bdb_tools.readers import build_reader +from bdb_tools.cupy_metrics import cupy_precision_score +from sklearn.metrics import roc_auc_score +import cupy as cp + +from dask_sql import Context + +# Logistic Regression params +# solver = "LBFGS" Used by passing `penalty=None` or "l2" +# step_size = 1 Not used +# numCorrections = 10 Not used +iterations = 100 +C = 10_000 # reg_lambda = 0 hence C for model is a large value +convergence_tol = 1e-9 + +wcs_columns = ["wcs_item_sk", "wcs_user_sk"] +items_columns = ["i_item_sk", "i_category", "i_category_id"] +customer_columns = ["c_customer_sk", "c_current_cdemo_sk"] +customer_dem_columns = ["cd_demo_sk", "cd_gender", "cd_education_status"] + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_ddf = table_reader.read("item", relevant_cols=items_columns, index=False) + customer_ddf = table_reader.read( + "customer", relevant_cols=customer_columns, index=False + ) + customer_dem_ddf = table_reader.read( + "customer_demographics", relevant_cols=customer_dem_columns, index=False + ) + + wcs_ddf = table_reader.read( + "web_clickstreams", relevant_cols=wcs_columns, index=False + ) + + bc.create_table("web_clickstreams", wcs_ddf) + bc.create_table("customer", customer_ddf) + bc.create_table("item", item_ddf) + bc.create_table("customer_demographics", customer_dem_ddf) + + # bc.create_table("web_clickstreams", os.path.join(data_dir, "web_clickstreams/*.parquet")) + # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + # bc.create_table( + # "customer_demographics", os.path.join(data_dir, "customer_demographics/*.parquet" + # )) + + +def build_and_predict_model(ml_input_df): + """ + Create a standardized feature matrix X and target array y. + Returns the model and accuracy statistics + """ + import cuml + from cuml.metrics import confusion_matrix + + feature_names = ["college_education", "male"] + [ + "clicks_in_%d" % i for i in range(1, 8) + ] + X = ml_input_df[feature_names] + # Standardize input matrix + X = (X - X.mean()) / X.std() + y = ml_input_df["clicks_in_category"] + + model = cuml.LogisticRegression( + tol=convergence_tol, + penalty="none", + solver="qn", + fit_intercept=True, + max_iter=iterations, + C=C, + ) + model.fit(X, y) + # + # Predict and evaluate accuracy + # (Should be 1.0) at SF-1 + # + results_dict = {} + y_pred = model.predict(X) + + results_dict["auc"] = roc_auc_score(y.to_array(), y_pred.to_array()) + results_dict["precision"] = cupy_precision_score(cp.asarray(y), cp.asarray(y_pred)) + results_dict["confusion_matrix"] = confusion_matrix( + cp.asarray(y, dtype="int32"), cp.asarray(y_pred, dtype="int32") + ) + results_dict["output_type"] = "supervised" + return results_dict + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = """ + SELECT + --wcs_user_sk, + clicks_in_category, + CASE WHEN cd_education_status IN ('Advanced Degree', 'College', '4 yr Degree', '2 yr Degree') + THEN 1 ELSE 0 END AS college_education, + CASE WHEN cd_gender = 'M' THEN 1 ELSE 0 END AS male, + clicks_in_1, + clicks_in_2, + clicks_in_3, + clicks_in_4, + clicks_in_5, + clicks_in_6, + clicks_in_7 + FROM + ( + SELECT + wcs_user_sk, + SUM( CASE WHEN i_category = 'Books' THEN 1 ELSE 0 END) AS clicks_in_category, + SUM( CASE WHEN i_category_id = 1 THEN 1 ELSE 0 END) AS clicks_in_1, + SUM( CASE WHEN i_category_id = 2 THEN 1 ELSE 0 END) AS clicks_in_2, + SUM( CASE WHEN i_category_id = 3 THEN 1 ELSE 0 END) AS clicks_in_3, + SUM( CASE WHEN i_category_id = 4 THEN 1 ELSE 0 END) AS clicks_in_4, + SUM( CASE WHEN i_category_id = 5 THEN 1 ELSE 0 END) AS clicks_in_5, + SUM( CASE WHEN i_category_id = 6 THEN 1 ELSE 0 END) AS clicks_in_6, + SUM( CASE WHEN i_category_id = 7 THEN 1 ELSE 0 END) AS clicks_in_7 + FROM web_clickstreams + INNER JOIN item it ON + ( + wcs_item_sk = i_item_sk + AND wcs_user_sk IS NOT NULL + ) + GROUP BY wcs_user_sk + ) q05_user_clicks_in_cat + INNER JOIN customer ct ON wcs_user_sk = c_customer_sk + INNER JOIN customer_demographics ON c_current_cdemo_sk = cd_demo_sk + """ + + cust_and_clicks_ddf = bc.sql(query) + + cust_and_clicks_ddf = cust_and_clicks_ddf.repartition(npartitions=1) + + # Convert clicks_in_category to a binary label + cust_and_clicks_ddf["clicks_in_category"] = ( + cust_and_clicks_ddf["clicks_in_category"] + > cust_and_clicks_ddf["clicks_in_category"].mean() + ).astype("int64") + + # Converting the dataframe to float64 as cuml logistic reg requires this + ml_input_df = cust_and_clicks_ddf.astype("float64") + + ml_input_df = ml_input_df.persist() + wait(ml_input_df) + + ml_tasks = [delayed(build_and_predict_model)(df) for df in ml_input_df.to_delayed()] + results_dict = client.compute(*ml_tasks, sync=True) + + return results_dict + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py new file mode 100755 index 00000000..69b8abe7 --- /dev/null +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -0,0 +1,163 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +# -------- Q6 ----------- +q06_LIMIT = 100 +# --web_sales and store_sales date +q06_YEAR = 2001 + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + web_sales_cols = [ + "ws_bill_customer_sk", + "ws_sold_date_sk", + "ws_ext_list_price", + "ws_ext_wholesale_cost", + "ws_ext_discount_amt", + "ws_ext_sales_price", + ] + store_sales_cols = [ + "ss_customer_sk", + "ss_sold_date_sk", + "ss_ext_list_price", + "ss_ext_wholesale_cost", + "ss_ext_discount_amt", + "ss_ext_sales_price", + ] + date_cols = ["d_date_sk", "d_year", "d_moy"] + customer_cols = [ + "c_customer_sk", + "c_customer_id", + "c_email_address", + "c_first_name", + "c_last_name", + "c_preferred_cust_flag", + "c_birth_country", + "c_login", + ] + + ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) + ss_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_df = table_reader.read("date_dim", relevant_cols=date_cols) + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + + bc.create_table('web_sales', ws_df) + bc.create_table('store_sales', ss_df) + bc.create_table('date_dim', date_df) + bc.create_table('customer', customer_df) + + # bc.create_table('web_sales', os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table('store_sales', os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table('customer', os.path.join(data_dir, "customer/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + WITH temp_table_1 as + ( + SELECT ss_customer_sk AS customer_sk, + sum( case when (d_year = {q06_YEAR}) THEN (((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2.0) ELSE 0.0 END) + AS first_year_total, + sum( case when (d_year = {q06_YEAR + 1}) THEN (((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2.0) ELSE 0.0 END) + AS second_year_total + FROM store_sales, + date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_year BETWEEN {q06_YEAR} AND {q06_YEAR + 1} + GROUP BY ss_customer_sk + -- first_year_total is an aggregation, rewrite all sum () statement + HAVING sum( case when (d_year = {q06_YEAR}) THEN (((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2.0) ELSE 0.0 END) > 0.0 + ), + temp_table_2 AS + ( + SELECT ws_bill_customer_sk AS customer_sk , + sum( case when (d_year = {q06_YEAR}) THEN (((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2.0) ELSE 0.0 END) + AS first_year_total, + sum( case when (d_year = {q06_YEAR + 1}) THEN (((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2.0) ELSE 0.0 END) + AS second_year_total + FROM web_sales, + date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_year BETWEEN {q06_YEAR} AND {q06_YEAR + 1} + GROUP BY ws_bill_customer_sk + -- required to avoid division by 0, because later we will divide by this value + HAVING sum( case when (d_year = {q06_YEAR}) THEN (((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2.0)ELSE 0.0 END) > 0.0 + ) + -- MAIN QUERY + SELECT + CAST( (web.second_year_total / web.first_year_total) AS DOUBLE) AS web_sales_increase_ratio, + c_customer_sk, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address + FROM temp_table_1 store, + temp_table_2 web, + customer c + WHERE store.customer_sk = web.customer_sk + AND web.customer_sk = c_customer_sk + -- if customer has sales in first year for both store and websales, + -- select him only if web second_year_total/first_year_total + -- ratio is bigger then his store second_year_total/first_year_total ratio. + AND (web.second_year_total / web.first_year_total) > + (store.second_year_total / store.first_year_total) + ORDER BY + web_sales_increase_ratio DESC, + c_customer_sk, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login + LIMIT {q06_LIMIT} + """ + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, bc = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) + diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py new file mode 100755 index 00000000..f49f75cb --- /dev/null +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -0,0 +1,118 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +from dask.distributed import Client +import os + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_cols = ["i_item_sk", "i_current_price", "i_category"] + store_sales_cols = ["ss_item_sk", "ss_customer_sk", "ss_sold_date_sk"] + store_cols = ["s_store_sk"] + date_cols = ["d_date_sk", "d_year", "d_moy"] + customer_cols = ["c_customer_sk", "c_current_addr_sk"] + customer_address_cols = ["ca_address_sk", "ca_state"] + + item_df = table_reader.read("item", relevant_cols=item_cols) + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + # store_df = table_reader.read("store", relevant_cols=store_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + customer_address_df = table_reader.read( + "customer_address", relevant_cols=customer_address_cols + ) + + bc.create_table("item", item_df) + bc.create_table("customer", customer_df) + bc.create_table("store_sales", store_sales_df) + bc.create_table("date_dim", date_dim_df) + bc.create_table("customer_address", customer_address_df) + + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table("customer_address", os.path.join(data_dir, "customer_address/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = """ + WITH temp_table as + ( + SELECT k.i_item_sk + FROM item k, + ( + SELECT i_category, + SUM(j.i_current_price) / COUNT(j.i_current_price) * 1.2 AS avg_price + FROM item j + GROUP BY j.i_category + ) avgCategoryPrice + WHERE avgCategoryPrice.i_category = k.i_category + AND k.i_current_price > avgCategoryPrice.avg_price + ) + SELECT ca_state, COUNT(*) AS cnt + FROM + customer_address a, + customer c, + store_sales s, + temp_table highPriceItems + WHERE a.ca_address_sk = c.c_current_addr_sk + AND c.c_customer_sk = s.ss_customer_sk + AND ca_state IS NOT NULL + AND ss_item_sk = highPriceItems.i_item_sk + AND s.ss_sold_date_sk IN + ( + SELECT d_date_sk + FROM date_dim + WHERE d_year = 2004 + AND d_moy = 7 + ) + GROUP BY ca_state + HAVING COUNT(*) >= 10 + ORDER BY cnt DESC, ca_state + LIMIT 10 + """ + + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, bc = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py new file mode 100755 index 00000000..878d5767 --- /dev/null +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -0,0 +1,281 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster +import cupy as cp +import numpy as np +import cudf + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q8 ----------- +q08_SECONDS_BEFORE_PURCHASE = 259200 +q08_STARTDATE = "2001-09-02" +q08_ENDDATE = "2002-09-02" + +REVIEW_CAT_CODE = 6 +NA_FLAG = 0 + + +def get_session_id_from_session_boundary(session_change_df, last_session_len): + """ + This function returns session starts given a session change df + """ + import cudf + + user_session_ids = session_change_df.tstamp_inSec + + ### up shift the session length df + session_len = session_change_df["t_index"].diff().reset_index(drop=True) + session_len = session_len.shift(-1) + + try: + session_len.iloc[-1] = last_session_len + except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 + session_len = cudf.Series([]) + + session_id_final_series = ( + cudf.Series(user_session_ids).repeat(session_len).reset_index(drop=True) + ) + return session_id_final_series + + +def get_session_id(df): + """ + This function creates a session id column for each click + The session id grows in incremeant for each user's susbequent session + Session boundry is defined by the time_out + """ + + df["user_change_flag"] = df["wcs_user_sk"].diff(periods=1) != 0 + df["user_change_flag"] = df["user_change_flag"].fillna(True) + df["session_change_flag"] = df["review_flag"] | df["user_change_flag"] + + df = df.reset_index(drop=True) + df["t_index"] = cp.arange(start=0, stop=len(df), dtype=np.int32) + + session_change_df = df[df["session_change_flag"]].reset_index(drop=True) + try: + last_session_len = len(df) - session_change_df["t_index"].iloc[-1] + except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 + last_session_len = 0 + + session_ids = get_session_id_from_session_boundary( + session_change_df, last_session_len + ) + + assert len(session_ids) == len(df) + return session_ids + + +def get_sessions(df): + df = df.sort_values( + by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] + ).reset_index(drop=True) + df["session_id"] = get_session_id(df) + return df + + +def get_unique_sales_keys_from_sessions(sessionized, review_cat_code): + sessionized["relevant"] = ( + (sessionized.tstamp_inSec - sessionized.session_id) + <= q08_SECONDS_BEFORE_PURCHASE + ) & (sessionized.wcs_sales_sk != NA_FLAG) + unique_sales_sk = ( + sessionized.query(f"wcs_sales_sk != {NA_FLAG}") + .query("relevant == True") + .query(f"wp_type_codes != {review_cat_code}") + .wcs_sales_sk.unique() + ) + + return unique_sales_sk + + +def prep_for_sessionization(df, review_cat_code): + df = df.fillna(NA_FLAG) + df = df.sort_values( + by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] + ).reset_index(drop=True) + + review_df = df.loc[df["wp_type_codes"] == review_cat_code] + # per user, the index of the first review + # need this to decide if a review was "recent enough" + every_users_first_review = ( + review_df[["wcs_user_sk", "tstamp_inSec"]] + .drop_duplicates() + .reset_index() + .groupby("wcs_user_sk")["index"] + .min() + .reset_index() + ) + every_users_first_review.columns = ["wcs_user_sk", "first_review_index"] + + # then reset the index to keep the old index before parallel join + df_merged = df.reset_index().merge( + every_users_first_review, how="left", on="wcs_user_sk" + ) + df_filtered = df_merged.query("index >= first_review_index") + return df_filtered + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + date_dim_cols = ["d_date_sk", "d_date"] + web_page_cols = ["wp_web_page_sk", "wp_type"] + web_sales_cols = ["ws_net_paid", "ws_order_number", "ws_sold_date_sk"] + + date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) + web_page_df = table_reader.read("web_page", relevant_cols=web_page_cols) + web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) + wcs_df = table_reader.read("web_clickstreams") + + bc.create_table("web_clickstreams", wcs_df) + bc.create_table("web_sales", web_sales_df) + bc.create_table("web_page", web_page_df) + bc.create_table("date_dim", date_dim_df) + + # bc.create_table("web_clickstreams", os.path.join(data_dir, "web_clickstreams/*.parquet")) + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table("web_page", os.path.join(data_dir, "web_page/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = f""" + SELECT d_date_sk + FROM date_dim + WHERE CAST(d_date as date) IN (date '{q08_STARTDATE}', + date '{q08_ENDDATE}') + ORDER BY CAST(d_date as date) asc + """ + result_dates_sk_filter = bc.sql(query_1).compute() + + # because `result_dates_sk_filter` has repetitive index + result_dates_sk_filter.index = list(range(0, result_dates_sk_filter.shape[0])) + q08_start_dt = result_dates_sk_filter['d_date_sk'][0] + q08_end_dt = result_dates_sk_filter['d_date_sk'][1] + + query_aux = """ + SELECT + wp_web_page_sk, + wp_type + FROM web_page + """ + web_page_df = bc.sql(query_aux) + + # cast to minimum viable dtype + web_page_df["wp_type"] = web_page_df["wp_type"].map_partitions( + lambda ser: ser.astype("category") + ) + + cpu_categories = web_page_df["wp_type"].compute().cat.categories.to_pandas() + REVIEW_CAT_CODE = cpu_categories.get_loc("review") + + codes_min_signed_type = cudf.utils.dtypes.min_signed_type(len(cpu_categories)) + + web_page_df["wp_type_codes"] = web_page_df["wp_type"].cat.codes.astype( + codes_min_signed_type + ) + + web_page_newcols = ["wp_web_page_sk", "wp_type_codes"] + web_page_df = web_page_df[web_page_newcols] + + web_page_df = web_page_df.persist() + wait(web_page_df) + bc.create_table('web_page_2', web_page_df) + + query_2 = f""" + SELECT + CAST(wcs_user_sk AS INTEGER) AS wcs_user_sk, + (wcs_click_date_sk * 86400 + wcs_click_time_sk) AS tstamp_inSec, + wcs_sales_sk, + wp_type_codes + FROM web_clickstreams + INNER JOIN web_page_2 ON wcs_web_page_sk = wp_web_page_sk + WHERE wcs_user_sk IS NOT NULL + AND wcs_click_date_sk BETWEEN {q08_start_dt} AND {q08_end_dt} + --in the future we want to remove this ORDER BY + ORDER BY wcs_user_sk + """ + merged_df = bc.sql(query_2) + + bc.drop_table("web_page_2") + del web_page_df + + merged_df = merged_df.shuffle(on=["wcs_user_sk"]) + merged_df["review_flag"] = merged_df.wp_type_codes == REVIEW_CAT_CODE + + prepped = merged_df.map_partitions( + prep_for_sessionization, review_cat_code=REVIEW_CAT_CODE + ) + + sessionized = prepped.map_partitions(get_sessions) + + unique_review_sales = sessionized.map_partitions( + get_unique_sales_keys_from_sessions, review_cat_code=REVIEW_CAT_CODE + ) + + unique_review_sales = unique_review_sales.to_frame() + + unique_review_sales = unique_review_sales.persist() + wait(unique_review_sales) + bc.create_table("reviews", unique_review_sales) + last_query = f""" + SELECT + CAST(review_total AS BIGINT) AS q08_review_sales_amount, + CAST(total - review_total AS BIGINT) AS no_q08_review_sales_amount + FROM + ( + SELECT + SUM(ws_net_paid) AS total, + SUM(CASE when wcs_sales_sk IS NULL THEN 0 ELSE 1 END * ws_net_paid) AS review_total + FROM web_sales + LEFT OUTER JOIN reviews ON ws_order_number = wcs_sales_sk + WHERE ws_sold_date_sk between {q08_start_dt} AND {q08_end_dt} + ) + """ + result = bc.sql(last_query) + + bc.drop_table("reviews") + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py new file mode 100755 index 00000000..d13dc302 --- /dev/null +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -0,0 +1,183 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +import os + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +# -------- Q9 ----------- +q09_year = 2001 + +q09_part1_ca_country = "United States" +q09_part1_ca_state_IN = "'KY', 'GA', 'NM'" +q09_part1_net_profit_min = 0 +q09_part1_net_profit_max = 2000 +q09_part1_education_status = "4 yr Degree" +q09_part1_marital_status = "M" +q09_part1_sales_price_min = 100 +q09_part1_sales_price_max = 150 + +q09_part2_ca_country = "United States" +q09_part2_ca_state_IN = "'MT', 'OR', 'IN'" +q09_part2_net_profit_min = 150 +q09_part2_net_profit_max = 3000 +q09_part2_education_status = "4 yr Degree" +q09_part2_marital_status = "M" +q09_part2_sales_price_min = 50 +q09_part2_sales_price_max = 200 + +q09_part3_ca_country = "United States" +q09_part3_ca_state_IN = "'WI', 'MO', 'WV'" +q09_part3_net_profit_min = 50 +q09_part3_net_profit_max = 25000 +q09_part3_education_status = "4 yr Degree" +q09_part3_marital_status = "M" +q09_part3_sales_price_min = 150 +q09_part3_sales_price_max = 200 + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ss_columns = [ + "ss_quantity", + "ss_sold_date_sk", + "ss_addr_sk", + "ss_store_sk", + "ss_cdemo_sk", + "ss_sales_price", + "ss_net_profit", + ] + + store_sales = table_reader.read("store_sales", relevant_cols=ss_columns) + + ca_columns = ["ca_address_sk", "ca_country", "ca_state"] + customer_address = table_reader.read("customer_address", relevant_cols=ca_columns) + + cd_columns = ["cd_demo_sk", "cd_marital_status", "cd_education_status"] + customer_demographics = table_reader.read( + "customer_demographics", relevant_cols=cd_columns + ) + + dd_columns = ["d_year", "d_date_sk"] + date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) + + s_columns = ["s_store_sk"] + store = table_reader.read("store", relevant_cols=s_columns) + + bc.create_table("store_sales", store_sales) + bc.create_table("customer_address", customer_address) + bc.create_table("customer_demographics", customer_demographics) + bc.create_table("date_dim", date_dim) + bc.create_table("store", store) + + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("customer_address", os.path.join(data_dir, "customer_address/*.parquet")) + # bc.create_table( + # "customer_demographics", os.path.join(data_dir, "customer_demographics/*.parquet" + # )) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + SELECT SUM(ss1.ss_quantity) + FROM store_sales ss1, + date_dim dd,customer_address ca1, + store s, + customer_demographics cd + -- select date range + WHERE ss1.ss_sold_date_sk = dd.d_date_sk + AND dd.d_year = {q09_year} + AND ss1.ss_addr_sk = ca1.ca_address_sk + AND s.s_store_sk = ss1.ss_store_sk + AND cd.cd_demo_sk = ss1.ss_cdemo_sk + AND + ( + ( + cd.cd_marital_status = '{q09_part1_marital_status}' + AND cd.cd_education_status = '{q09_part1_education_status}' + AND {q09_part1_sales_price_min} <= ss1.ss_sales_price + AND ss1.ss_sales_price <= {q09_part1_sales_price_max} + ) + OR + ( + cd.cd_marital_status = '{q09_part2_marital_status}' + AND cd.cd_education_status = '{q09_part2_education_status}' + AND {q09_part2_sales_price_min} <= ss1.ss_sales_price + AND ss1.ss_sales_price <= {q09_part2_sales_price_max} + ) + OR + ( + cd.cd_marital_status = '{q09_part3_marital_status}' + AND cd.cd_education_status = '{q09_part3_education_status}' + AND {q09_part3_sales_price_min} <= ss1.ss_sales_price + AND ss1.ss_sales_price <= {q09_part3_sales_price_max} + ) + ) + AND + ( + ( + ca1.ca_country = '{q09_part1_ca_country}' + AND ca1.ca_state IN ({q09_part1_ca_state_IN}) + AND {q09_part1_net_profit_min} <= ss1.ss_net_profit + AND ss1.ss_net_profit <= {q09_part1_net_profit_max} + ) + OR + ( + ca1.ca_country = '{q09_part2_ca_country}' + AND ca1.ca_state IN ({q09_part2_ca_state_IN}) + AND {q09_part2_net_profit_min} <= ss1.ss_net_profit + AND ss1.ss_net_profit <= {q09_part2_net_profit_max} + ) + OR + ( + ca1.ca_country = '{q09_part3_ca_country}' + AND ca1.ca_state IN ({q09_part3_ca_state_IN}) + AND {q09_part3_net_profit_min} <= ss1.ss_net_profit + AND ss1.ss_net_profit <= {q09_part3_net_profit_max} + ) + ) + """ + result = bc.sql(query) + result.columns = ["sum(ss_quantity)"] + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py new file mode 100755 index 00000000..da1cb162 --- /dev/null +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -0,0 +1,170 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +import dask_cudf + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.text import ( + create_sentences_from_reviews, + create_words_from_sentences +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +eol_char = "รจ" + + +def read_tables(data_dir, bc): + + ### splitting by row groups for better parallelism + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] + + product_reviews_df = table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols, + ) + + bc.create_table("product_reviews", product_reviews_df) + + # bc.create_table('product_reviews', os.path.join(data_dir, "product_reviews/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = """ + SELECT pr_item_sk, + pr_review_content, + pr_review_sk + FROM product_reviews + where pr_review_content IS NOT NULL + ORDER BY pr_item_sk, pr_review_content, pr_review_sk + """ + product_reviews_df = bc.sql(query_1) + + product_reviews_df[ + "pr_review_content" + ] = product_reviews_df.pr_review_content.str.lower() + product_reviews_df[ + "pr_review_content" + ] = product_reviews_df.pr_review_content.str.replace( + [".", "?", "!"], [eol_char], regex=False + ) + + sentences = product_reviews_df.map_partitions(create_sentences_from_reviews) + + product_reviews_df = product_reviews_df[["pr_item_sk", "pr_review_sk"]] + product_reviews_df["pr_review_sk"] = product_reviews_df["pr_review_sk"].astype("int32") + + # need the global position in the sentence tokenized df + sentences["x"] = 1 + sentences["sentence_tokenized_global_pos"] = sentences.x.cumsum() + del sentences["x"] + + word_df = sentences.map_partitions( + create_words_from_sentences, + global_position_column="sentence_tokenized_global_pos", + ) + + product_reviews_df = product_reviews_df.persist() + wait(product_reviews_df) + bc.create_table('product_reviews_df', product_reviews_df) + + sentences = sentences.persist() + wait(sentences) + bc.create_table('sentences', sentences) + + # These files come from the official TPCx-BB kit + # We extracted them from bigbenchqueriesmr.jar + # Need to pass the absolute path for these txt files + sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") + ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) + bc.create_table('negative_sentiment', ns_df) + ps_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "positiveSentiment.txt"), names=["sentiment_word"]) + bc.create_table('positive_sentiment', ps_df) + + word_df = word_df.persist() + wait(word_df) + bc.create_table('word_df', word_df) + + query = ''' + SELECT pr_item_sk as item_sk, + sentence as review_sentence, + sentiment, + sentiment_word FROM + ( + SELECT review_idx_global_pos, + sentiment_word, + sentiment, + sentence FROM + ( + WITH sent_df AS + ( + (SELECT sentiment_word, 'POS' as sentiment + FROM positive_sentiment + GROUP BY sentiment_word) + UNION ALL + (SELECT sentiment_word, 'NEG' as sentiment + FROM negative_sentiment + GROUP BY sentiment_word) + ) + SELECT * FROM word_df + INNER JOIN sent_df + ON word_df.word = sent_df.sentiment_word + ) word_sentence_sentiment + LEFT JOIN sentences + ON word_sentence_sentiment.sentence_idx_global_pos = sentences.sentence_tokenized_global_pos + ) temp + INNER JOIN product_reviews_df + ON temp.review_idx_global_pos = product_reviews_df.pr_review_sk + ORDER BY item_sk, review_sentence, sentiment, sentiment_word + ''' + result = bc.sql(query) + + bc.drop_table("product_reviews_df") + del product_reviews_df + bc.drop_table("sentences") + del sentences + bc.drop_table("word_df") + del word_df + + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py new file mode 100755 index 00000000..bece80ff --- /dev/null +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -0,0 +1,108 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +import os +import cudf + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + product_review_cols = [ + "pr_review_rating", + "pr_item_sk", + ] + web_sales_cols = [ + "ws_sold_date_sk", + "ws_net_paid", + "ws_item_sk", + ] + date_cols = ["d_date_sk", "d_date"] + + pr_df = table_reader.read("product_reviews", relevant_cols=product_review_cols) + # we only read int columns here so it should scale up to sf-10k as just 26M rows + pr_df = pr_df.repartition(npartitions=1) + + ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) + date_df = table_reader.read("date_dim", relevant_cols=date_cols) + + bc.create_table("web_sales", ws_df) + bc.create_table("product_reviews", pr_df) + bc.create_table("date_dim", date_df) + + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = """ + WITH p AS + ( + SELECT + pr_item_sk, + count(pr_item_sk) AS r_count, + AVG( CAST(pr_review_rating AS DOUBLE) ) avg_rating + FROM product_reviews + WHERE pr_item_sk IS NOT NULL + GROUP BY pr_item_sk + ), s AS + ( + SELECT + ws_item_sk + FROM web_sales ws + INNER JOIN date_dim d ON ws.ws_sold_date_sk = d.d_date_sk + WHERE ws_item_sk IS NOT null + AND CAST(d.d_date AS DATE) >= DATE '2003-01-02' + AND CAST(d.d_date AS DATE) <= DATE '2003-02-02' + GROUP BY ws_item_sk + ) + SELECT p.r_count AS x, + p.avg_rating AS y + FROM s INNER JOIN p ON p.pr_item_sk = s.ws_item_sk + """ + + result = bc.sql(query) + sales_corr = result["x"].corr(result["y"]).compute() + result_df = cudf.DataFrame([sales_corr]) + result_df.columns = ["corr(CAST(reviews_count AS DOUBLE), avg_rating)"] + return result_df + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py new file mode 100755 index 00000000..85bc01ec --- /dev/null +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -0,0 +1,101 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +# -------- Q12 ----------- +q12_i_category_IN = "'Books', 'Electronics'" + +item_cols = ["i_item_sk", "i_category"] +store_sales_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_customer_sk"] + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_df = table_reader.read("item", relevant_cols=item_cols) + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + wcs_df = table_reader.read("web_clickstreams") + + bc.create_table("web_clickstreams", wcs_df) + bc.create_table("store_sales", store_sales_df) + bc.create_table("item", item_df) + + # bc.create_table("web_clickstreams", + # os.path.join(data_dir, "web_clickstreams/*.parquet")) + # bc.create_table("store_sales", + # os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + SELECT DISTINCT wcs_user_sk + FROM + ( + SELECT DISTINCT + wcs_user_sk, + wcs_click_date_sk + FROM web_clickstreams, item + WHERE wcs_click_date_sk BETWEEN 37134 AND 37164 + AND i_category IN ({q12_i_category_IN}) + AND wcs_item_sk = i_item_sk + AND wcs_user_sk IS NOT NULL + AND wcs_sales_sk IS NULL + ) webInRange, + ( + SELECT DISTINCT + ss_customer_sk, + ss_sold_date_sk + FROM store_sales, item + WHERE ss_sold_date_sk BETWEEN 37134 AND 37224 + AND i_category IN ({q12_i_category_IN}) -- filter given category + AND ss_item_sk = i_item_sk + AND ss_customer_sk IS NOT NULL + ) storeInRange + WHERE wcs_user_sk = ss_customer_sk + AND wcs_click_date_sk < ss_sold_date_sk + ORDER BY wcs_user_sk + """ + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py new file mode 100644 index 00000000..71109386 --- /dev/null +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -0,0 +1,139 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +from dask.distributed import Client +import os + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + date_cols = ["d_date_sk", "d_year"] + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + + customer_cols = ["c_customer_sk", "c_customer_id", "c_first_name", "c_last_name"] + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + + s_sales_cols = ["ss_sold_date_sk", "ss_customer_sk", "ss_net_paid"] + s_sales_df = table_reader.read("store_sales", relevant_cols=s_sales_cols) + + w_sales_cols = ["ws_sold_date_sk", "ws_bill_customer_sk", "ws_net_paid"] + web_sales_df = table_reader.read("web_sales", relevant_cols=w_sales_cols) + + bc.create_table("date_dim", date_dim_df) + bc.create_table("customer", customer_df) + bc.create_table("store_sales", s_sales_df) + bc.create_table("web_sales", web_sales_df) + + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = """ + SELECT + ss.ss_customer_sk AS customer_sk, + sum( case when (d_year = 2001) THEN ss_net_paid ELSE 0.0 END) first_year_total, + sum( case when (d_year = 2002) THEN ss_net_paid ELSE 0.0 END) second_year_total + FROM store_sales ss + JOIN + ( + SELECT d_date_sk, d_year + FROM date_dim d + WHERE d.d_year in (2001, 2002) + ) dd on ( ss.ss_sold_date_sk = dd.d_date_sk ) + GROUP BY ss.ss_customer_sk + HAVING sum( case when (d_year = 2001) THEN ss_net_paid ELSE 0.0 END) > 0.0 + """ + temp_table1 = bc.sql(query_1) + + temp_table1 = temp_table1.persist() + wait(temp_table1) + bc.create_table("temp_table1", temp_table1) + query_2 = """ + SELECT + ws.ws_bill_customer_sk AS customer_sk, + sum( case when (d_year = 2001) THEN ws_net_paid ELSE 0.0 END) first_year_total, + sum( case when (d_year = 2002) THEN ws_net_paid ELSE 0.0 END) second_year_total + FROM web_sales ws + JOIN + ( + SELECT d_date_sk, d_year + FROM date_dim d + WHERE d.d_year in (2001, 2002) + ) dd ON ( ws.ws_sold_date_sk = dd.d_date_sk ) + GROUP BY ws.ws_bill_customer_sk + HAVING sum( case when (d_year = 2001) THEN ws_net_paid ELSE 0.0 END) > 0.0 + """ + temp_table2 = bc.sql(query_2) + + temp_table2 = temp_table2.persist() + wait(temp_table2) + bc.create_table("temp_table2", temp_table2) + query = """ + SELECT + CAST(c_customer_sk AS BIGINT) as c_customer_sk, + c_first_name, + c_last_name, + (store.second_year_total / store.first_year_total) AS storeSalesIncreaseRatio, + (web.second_year_total / web.first_year_total) AS webSalesIncreaseRatio + FROM temp_table1 store, + temp_table2 web, + customer c + WHERE store.customer_sk = web.customer_sk + AND web.customer_sk = c_customer_sk + AND (web.second_year_total / web.first_year_total) > (store.second_year_total / store.first_year_total) + ORDER BY webSalesIncreaseRatio DESC, + c_customer_sk, + c_first_name, + c_last_name + LIMIT 100 + """ + result = bc.sql(query) + + bc.drop_table("temp_table1") + bc.drop_table("temp_table2") + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py new file mode 100755 index 00000000..3a1e5e25 --- /dev/null +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -0,0 +1,97 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +import os + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ws_columns = ["ws_ship_hdemo_sk", "ws_web_page_sk", "ws_sold_time_sk"] + web_sales = table_reader.read("web_sales", relevant_cols=ws_columns) + + hd_columns = ["hd_demo_sk", "hd_dep_count"] + household_demographics = table_reader.read( + "household_demographics", relevant_cols=hd_columns + ) + + wp_columns = ["wp_web_page_sk", "wp_char_count"] + web_page = table_reader.read("web_page", relevant_cols=wp_columns) + + td_columns = ["t_time_sk", "t_hour"] + time_dim = table_reader.read("time_dim", relevant_cols=td_columns) + + bc.create_table("household_demographics", household_demographics) + bc.create_table("web_page", web_page) + bc.create_table("web_sales", web_sales) + bc.create_table("time_dim", time_dim) + + # bc.create_table( + # "household_demographics", os.path.join(data_dir, "household_demographics/*.parquet" + # )) + # bc.create_table("web_page", os.path.join(data_dir, "web_page/*.parquet")) + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table("time_dim", os.path.join(data_dir, "time_dim/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = """ + SELECT CASE WHEN pmc > 0.0 THEN CAST (amc AS DOUBLE) / CAST (pmc AS DOUBLE) ELSE -1.0 END AS am_pm_ratio + FROM + ( + SELECT SUM(amc1) AS amc, SUM(pmc1) AS pmc + FROM + ( + SELECT + CASE WHEN t_hour BETWEEN 7 AND 8 THEN COUNT(1) ELSE 0 END AS amc1, + CASE WHEN t_hour BETWEEN 19 AND 20 THEN COUNT(1) ELSE 0 END AS pmc1 + FROM web_sales ws + JOIN household_demographics hd ON (hd.hd_demo_sk = ws.ws_ship_hdemo_sk and hd.hd_dep_count = 5) + JOIN web_page wp ON (wp.wp_web_page_sk = ws.ws_web_page_sk and wp.wp_char_count BETWEEN 5000 AND 6000) + JOIN time_dim td ON (td.t_time_sk = ws.ws_sold_time_sk and td.t_hour IN (7,8,19,20)) + GROUP BY t_hour + ) cnt_am_pm + ) sum_am_pm + """ + + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py new file mode 100755 index 00000000..b3be1c41 --- /dev/null +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -0,0 +1,102 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +import os + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +# -------- Q15 ----------- +# --store_sales date range +q15_startDate = "2001-09-02" +# --+1year +q15_endDate = "2002-09-02" +q15_store_sk = 10 + +store_sales_cols = ["ss_sold_date_sk", "ss_net_paid", "ss_store_sk", "ss_item_sk"] +date_cols = ["d_date", "d_date_sk"] +item_cols = ["i_item_sk", "i_category_id"] + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + + bc.create_table("store_sales", store_sales_df) + bc.create_table("date_dim", date_dim_df) + bc.create_table("item", item_df) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + SELECT * + FROM + ( + SELECT + cat, + ( (count(x) * SUM(xy) - SUM(x) * SUM(y)) / (count(x) * SUM(xx) - SUM(x) * SUM(x)) ) AS slope, + (SUM(y) - ((count(x) * SUM(xy) - SUM(x) * SUM(y)) / (count(x) * SUM(xx) - SUM(x)*SUM(x)) ) * SUM(x)) / count(x) AS intercept + FROM + ( + SELECT + i.i_category_id AS cat, + s.ss_sold_date_sk AS x, + CAST(SUM(s.ss_net_paid) AS DOUBLE) AS y, + CAST(s.ss_sold_date_sk * SUM(s.ss_net_paid) AS DOUBLE) AS xy, + CAST(s.ss_sold_date_sk * s.ss_sold_date_sk AS DOUBLE) AS xx + FROM store_sales s + INNER JOIN item i ON s.ss_item_sk = i.i_item_sk + INNER JOIN date_dim d ON s.ss_sold_date_sk = d.d_date_sk + WHERE s.ss_store_sk = {q15_store_sk} + AND i.i_category_id IS NOT NULL + AND CAST(d.d_date AS DATE) >= DATE '{q15_startDate}' + AND CAST(d.d_date AS DATE) <= DATE '{q15_endDate}' + GROUP BY i.i_category_id, s.ss_sold_date_sk + ) temp + GROUP BY cat + ) regression + WHERE slope <= 0.0 + ORDER BY cat + """ + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, bc = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py new file mode 100755 index 00000000..36f4059d --- /dev/null +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -0,0 +1,138 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from bdb_tools.cluster_startup import attach_to_cluster +import os + +import datetime +from datetime import timedelta +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +websale_cols = [ + "ws_order_number", + "ws_item_sk", + "ws_warehouse_sk", + "ws_sold_date_sk", + "ws_sales_price", +] +web_returns_cols = ["wr_order_number", "wr_item_sk", "wr_refunded_cash"] +date_cols = ["d_date", "d_date_sk"] +item_cols = ["i_item_sk", "i_item_id"] +warehouse_cols = ["w_warehouse_sk", "w_state"] + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) + web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + warehouse_df = table_reader.read("warehouse", relevant_cols=warehouse_cols) + + bc.create_table("web_sales", web_sales_df) + bc.create_table("web_returns", web_returns_df) + bc.create_table("date_dim", date_dim_df) + bc.create_table("item", item_df) + bc.create_table("warehouse", warehouse_df) + + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table("web_returns", os.path.join(data_dir, "web_returns/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + # bc.create_table("warehouse", os.path.join(data_dir, "warehouse/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + date = datetime.datetime(2001, 3, 16) + start = (date + timedelta(days=-30)).strftime("%Y-%m-%d") + end = (date + timedelta(days=30)).strftime("%Y-%m-%d") + mid = date.strftime("%Y-%m-%d") + + date_query = f""" + SELECT d_date_sk + FROM date_dim + WHERE CAST(d_date as DATE) IN (DATE '{start}', DATE '{mid}', DATE '{end}') + ORDER BY CAST(d_date as date) ASC + """ + + dates = bc.sql(date_query) + + cpu_dates = dates["d_date_sk"].compute().to_pandas() + cpu_dates.index = list(range(0, cpu_dates.shape[0])) + + last_query = f""" + SELECT w_state, i_item_id, + SUM + ( + CASE WHEN ws_sold_date_sk < {str(cpu_dates[1])} + THEN ws_sales_price - COALESCE(wr_refunded_cash,0) + ELSE 0.0 END + ) AS sales_before, + SUM + ( + CASE WHEN ws_sold_date_sk >= {str(cpu_dates[1])} + THEN ws_sales_price - COALESCE(wr_refunded_cash,0) + ELSE 0.0 END + ) AS sales_after + FROM + ( + SELECT ws_item_sk, + ws_warehouse_sk, + ws_sold_date_sk, + ws_sales_price, + wr_refunded_cash + FROM web_sales ws + LEFT OUTER JOIN web_returns wr ON + ( + ws.ws_order_number = wr.wr_order_number + AND ws.ws_item_sk = wr.wr_item_sk + ) + WHERE ws_sold_date_sk BETWEEN {str(cpu_dates[0])} + AND {str(cpu_dates[2])} + ) a1 + JOIN item i ON a1.ws_item_sk = i.i_item_sk + JOIN warehouse w ON a1.ws_warehouse_sk = w.w_warehouse_sk + GROUP BY w_state,i_item_id + ORDER BY w_state,i_item_id + LIMIT 100 + """ + + result = bc.sql(last_query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py new file mode 100755 index 00000000..7dca433d --- /dev/null +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -0,0 +1,141 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +# ------- Q17 ------ +q17_gmt_offset = -5.0 +# --store_sales date +q17_year = 2001 +q17_month = 12 +q17_i_category_IN = "'Books', 'Music'" + +store_sales_cols = [ + "ss_ext_sales_price", + "ss_sold_date_sk", + "ss_store_sk", + "ss_customer_sk", + "ss_promo_sk", + "ss_item_sk", +] +item_cols = ["i_category", "i_item_sk"] +customer_cols = ["c_customer_sk", "c_current_addr_sk"] +store_cols = ["s_gmt_offset", "s_store_sk"] +date_cols = ["d_date_sk", "d_year", "d_moy"] +customer_address_cols = ["ca_address_sk", "ca_gmt_offset"] +promotion_cols = ["p_channel_email", "p_channel_dmail", "p_channel_tv", "p_promo_sk"] + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + store_df = table_reader.read("store", relevant_cols=store_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + customer_address_df = table_reader.read( + "customer_address", relevant_cols=customer_address_cols + ) + promotion_df = table_reader.read("promotion", relevant_cols=promotion_cols) + + bc.create_table("store_sales", store_sales_df) + bc.create_table("item", item_df) + bc.create_table("customer", customer_df) + bc.create_table("store", store_df) + bc.create_table("date_dim", date_dim_df) + bc.create_table("customer_address", customer_address_df) + bc.create_table("promotion", promotion_df) + + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) + # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table("customer_address", os.path.join(data_dir, "customer_address/*.parquet")) + # bc.create_table("promotion", os.path.join(data_dir, "promotion/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_date = f""" + select min(d_date_sk) as min_d_date_sk, + max(d_date_sk) as max_d_date_sk + from date_dim + where d_year = {q17_year} + and d_moy = {q17_month} + """ + dates_result = bc.sql(query_date).compute() + + min_date_sk_val = dates_result["min_d_date_sk"][0] + max_date_sk_val = dates_result["max_d_date_sk"][0] + + query = f""" + SELECT sum(promotional) as promotional, + sum(total) as total, + CASE WHEN sum(total) > 0.0 THEN (100.0 * sum(promotional)) / sum(total) + ELSE 0.0 END as promo_percent + FROM + ( + SELECT p_channel_email, + p_channel_dmail, + p_channel_tv, + SUM( CAST(ss_ext_sales_price AS DOUBLE) ) total, + CASE WHEN (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') + THEN SUM(CAST(ss_ext_sales_price AS DOUBLE)) ELSE 0 END as promotional + FROM store_sales ss + INNER JOIN promotion p ON ss.ss_promo_sk = p.p_promo_sk + inner join item i on ss.ss_item_sk = i.i_item_sk + inner join store s on ss.ss_store_sk = s.s_store_sk + inner join customer c on c.c_customer_sk = ss.ss_customer_sk + inner join customer_address ca + on c.c_current_addr_sk = ca.ca_address_sk + WHERE i.i_category IN ({q17_i_category_IN}) + AND s.s_gmt_offset = {q17_gmt_offset} + AND ca.ca_gmt_offset = {q17_gmt_offset} + AND ss.ss_sold_date_sk >= {min_date_sk_val} + AND ss.ss_sold_date_sk <= {max_date_sk_val} + GROUP BY p_channel_email, p_channel_dmail, p_channel_tv + ) sum_promotional + -- we don't need a 'ON' join condition. result is just two numbers. + """ + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py new file mode 100755 index 00000000..b0c92ec9 --- /dev/null +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -0,0 +1,360 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster +import numpy as np +import cupy as cp + +from bdb_tools.text import create_sentences_from_reviews, create_words_from_sentences + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q18 ----------- +q18_startDate = "2001-05-02" +# --+90days +q18_endDate = "2001-09-02" + +EOL_CHAR = "รจ" + + +def create_found_reshaped_with_global_pos(found, targets): + """Given the dataframe created by mapping find_targets_in_reviews, + create a new dataframe in which the nonzero values in each row are exploded + to get their own row. Each row will contain the word, its mapping in the column order, + and the pr_review_sk for the review from which it came. + + Having these as two separate functions makes managing dask metadata easier. + """ + import cudf + + target_df = cudf.DataFrame({"word": targets}).reset_index(drop=False) + target_df.columns = ["word_mapping", "word"] + + df_clean = found.drop(["pr_review_sk"], axis=1) + + row_idxs, col_idxs = df_clean.values.nonzero() + + found_reshaped = cudf.DataFrame( + {"word_mapping": col_idxs, "pr_review_sk": found["pr_review_sk"].iloc[row_idxs]} + ) + found_reshaped = found_reshaped.merge(target_df, on="word_mapping", how="inner")[ + ["word", "pr_review_sk"] + ] + return found_reshaped + + +def find_targets_in_reviews_helper(ddf, targets, str_col_name="pr_review_content"): + """returns a N x K matrix, where N is the number of rows in ddf that + contain one of the target words and K is the number of words in targets. + + If a target word is found in a review, the value in that row, column + is non-zero. + + At the end, any row with non-zero values is returned. + + """ + import cudf + from cudf._lib.strings import find_multiple + + lowered = ddf[str_col_name].str.lower() + + ## TODO: Do the replace/any in cupy land before going to cuDF + resdf = cudf.DataFrame( + cp.asarray( + find_multiple.find_multiple(lowered._column, targets._column) + ).reshape(-1, len(targets)) + ) + + resdf = resdf.replace([0, -1], [1, 0]) + found_mask = resdf.any(axis=1) + resdf["pr_review_sk"] = ddf["pr_review_sk"] + found = resdf.loc[found_mask] + return create_found_reshaped_with_global_pos(found, targets) + + +def find_relevant_reviews(df, targets, str_col_name="pr_review_content"): + """ + This function finds the reviews containg target stores and returns the + relevant reviews + """ + import cudf + + targets = cudf.Series(targets) + targets_lower = targets.str.lower() + reviews_found = find_targets_in_reviews_helper(df, targets_lower)[ + ["word", "pr_review_sk"] + ] + + combined = reviews_found.merge( + df[["pr_review_date", "pr_review_sk"]], how="inner", on=["pr_review_sk"] + ) + + return combined + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + + store_sales_cols = [ + "ss_store_sk", + "ss_sold_date_sk", + "ss_net_paid", + ] + date_cols = ["d_date_sk", "d_date"] + store_cols = ["s_store_sk", "s_store_name"] + + store_sales = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_dim = table_reader.read("date_dim", relevant_cols=date_cols) + store = table_reader.read("store", relevant_cols=store_cols) + + ### splitting by row groups for better parallelism + pr_table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + + product_reviews_cols = ["pr_review_date", "pr_review_content", "pr_review_sk"] + product_reviews = pr_table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols, + ) + + bc.create_table("store", store) + bc.create_table("store_sales", store_sales) + bc.create_table("date_dim", date_dim) + bc.create_table("product_reviews", product_reviews) + + # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = f""" + WITH temp_table1 AS + ( + SELECT CAST(s.s_store_sk AS INTEGER) AS s_store_sk, + s.s_store_name , + CAST(s.s_store_sk AS VARCHAR) || '_' || s.s_store_name + AS store_ID + FROM store s, + ( + SELECT temp.ss_store_sk, + ((count(temp.x) * SUM(temp.xy) - SUM(temp.x) * SUM(temp.y)) + / (count(temp.x) * SUM(temp.xx) - SUM(temp.x) * SUM(temp.x)) + ) AS slope + FROM + ( + SELECT + s.ss_store_sk, + s.ss_sold_date_sk AS x, + CAST( SUM(s.ss_net_paid) AS DOUBLE) AS y, + s.ss_sold_date_sk * SUM(s.ss_net_paid) AS xy, + s.ss_sold_date_sk * s.ss_sold_date_sk AS xx + FROM store_sales s + WHERE EXISTS + ( + SELECT * -- d_date_sk + FROM date_dim d + WHERE s.ss_sold_date_sk = d.d_date_sk + AND CAST(d.d_date AS DATE) >= DATE '{q18_startDate}' + AND CAST(d.d_date AS DATE) <= DATE '{q18_endDate}' + ) + GROUP BY s.ss_store_sk, s.ss_sold_date_sk + ) temp + GROUP BY temp.ss_store_sk + ) regression_analysis + WHERE slope <= 0 --flat or declining sales + AND s.s_store_sk = regression_analysis.ss_store_sk + ) + SELECT * FROM temp_table1 + """ + stores_with_regression = bc.sql(query_1) + + query_2 = """ + SELECT pr_review_date, + pr_review_content, + CAST(pr_review_sk AS INTEGER) AS pr_review_sk + FROM product_reviews + WHERE pr_review_content IS NOT NULL + ORDER BY pr_review_date, pr_review_content, pr_review_sk + """ + no_nulls = bc.sql(query_2) + + targets = ( + stores_with_regression.s_store_name.str.lower() + .unique() + .compute() + .to_arrow() + .to_pylist() + ) + + # perssiting because no_nulls is used twice + no_nulls = no_nulls.persist() + + import cudf + + temp_table2_meta_empty_df = cudf.DataFrame( + { + "word": ["a"], + "pr_review_sk": np.ones(1, dtype=np.int64), + "pr_review_date": ["a"], + } + ).head(0) + + # get relevant reviews + combined = no_nulls.map_partitions( + find_relevant_reviews, targets, meta=temp_table2_meta_empty_df, + ) + + no_nulls["pr_review_content"] = no_nulls.pr_review_content.str.replace( + [". ", "? ", "! "], [EOL_CHAR], regex=False + ) + + stores_with_regression["store_ID"] = stores_with_regression.s_store_sk.astype( + "str" + ).str.cat(stores_with_regression.s_store_name, sep="_") + + stores_with_regression[ + "s_store_name" + ] = stores_with_regression.s_store_name.str.lower() + + stores_with_regression = stores_with_regression.persist() + wait(stores_with_regression) + bc.create_table("stores_with_regression", stores_with_regression) + + combined = combined.persist() + wait(combined) + bc.create_table("combined", combined) + + query_3 = """ + SELECT store_ID, + pr_review_date, + CAST(pr_review_sk AS INTEGER) AS pr_review_sk + FROM stores_with_regression + INNER JOIN combined ON s_store_name = word + """ + temp_table2 = bc.sql(query_3) + + bc.drop_table("stores_with_regression") + del stores_with_regression + + bc.drop_table("combined") + del combined + + # REAL QUERY + sentences = no_nulls.map_partitions(create_sentences_from_reviews) + + # need the global position in the sentence tokenized df + sentences["x"] = 1 + sentences["sentence_tokenized_global_pos"] = sentences.x.cumsum() + del sentences["x"] + + word_df = sentences.map_partitions( + create_words_from_sentences, + global_position_column="sentence_tokenized_global_pos", + ) + + # This txt file comes from the official TPCx-BB kit + # We extracted it from bigbenchqueriesmr.jar + # Need to pass the absolute path for this txt file + sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") + ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) + bc.create_table('negative_sentiment', ns_df) + + word_df = word_df.persist() + wait(word_df) + bc.create_table("word_df", word_df) + + sentences = sentences.persist() + wait(sentences) + bc.create_table("sentences", sentences) + + temp_table2 = temp_table2.persist() + wait(temp_table2) + bc.create_table("temp_table2", temp_table2) + + query_4 = """ + WITH sentences_table AS + ( + select sentence, + review_idx_global_pos, + CAST(sentence_tokenized_global_pos AS BIGINT) AS + sentence_tokenized_global_pos + from sentences + ), negativeSentiment AS + ( + SELECT DISTINCT sentiment_word AS word + FROM sent_df + ), word_sentence_sentiment AS + ( + SELECT n.word, + CAST(wd.sentence_idx_global_pos AS BIGINT) AS + sentence_idx_global_pos, + 'NEG' AS sentiment + FROM word_df wd + INNER JOIN negativeSentiment n ON wd.word = n.word + ), word_sentence_sentiment_with_sentence_info AS + ( + SELECT * FROM word_sentence_sentiment + LEFT JOIN sentences_table + ON sentence_idx_global_pos = sentence_tokenized_global_pos + ) + SELECT tt2.store_ID AS s_name, + tt2.pr_review_date AS r_date, + wsswsi.sentence AS r_sentence, + wsswsi.sentiment AS sentiment, + wsswsi.word AS sentiment_word + FROM word_sentence_sentiment_with_sentence_info wsswsi + INNER JOIN temp_table2 tt2 + ON wsswsi.review_idx_global_pos = tt2.pr_review_sk + ORDER BY s_name, r_date, r_sentence, sentiment_word + """ + result = bc.sql(query_4) + + bc.drop_table("word_df") + del word_df + bc.drop_table("sentences") + del sentences + bc.drop_table("temp_table2") + del temp_table2 + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py new file mode 100755 index 00000000..4e6c2b26 --- /dev/null +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -0,0 +1,212 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.text import ( + create_sentences_from_reviews, + create_words_from_sentences +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q19 ----------- +q19_returns_dates_IN = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] + +eol_char = "รจ" + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + date_dim_cols = ["d_week_seq", "d_date_sk", "d_date"] + date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) + store_returns_cols = ["sr_returned_date_sk", "sr_item_sk", "sr_return_quantity"] + store_returns_df = table_reader.read( + "store_returns", relevant_cols=store_returns_cols + ) + web_returns_cols = ["wr_returned_date_sk", "wr_item_sk", "wr_return_quantity"] + web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) + + ### splitting by row groups for better parallelism + pr_table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + + product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] + product_reviews_df = pr_table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols + ) + + bc.create_table('web_returns', web_returns_df) + bc.create_table('date_dim', date_dim_df) + bc.create_table('product_reviews', product_reviews_df) + bc.create_table('store_returns', store_returns_df) + + # bc.create_table('web_returns', os.path.join(data_dir, "web_returns/*.parquet")) + # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table('product_reviews', os.path.join(data_dir, "product_reviews/*.parquet")) + # bc.create_table('store_returns', os.path.join(data_dir, "store_returns/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + WITH dateFilter AS + ( + -- within the week ending a given date + SELECT d1.d_date_sk + FROM date_dim d1, date_dim d2 + WHERE d1.d_week_seq = d2.d_week_seq + AND CAST(d2.d_date AS DATE) IN (DATE '{q19_returns_dates_IN[0]}', + DATE '{q19_returns_dates_IN[1]}', + DATE '{q19_returns_dates_IN[2]}', + DATE '{q19_returns_dates_IN[3]}') + ), fsr AS + ( + --store returns in week ending given date + SELECT sr_item_sk, SUM(sr_return_quantity) sr_item_qty + FROM store_returns sr + INNER JOIN dateFilter d + ON sr.sr_returned_date_sk = d.d_date_sk + GROUP BY sr_item_sk --across all store and web channels + HAVING SUM(sr_return_quantity) > 0 + ), fwr AS + ( + --web returns in week ending given date + SELECT wr_item_sk, SUM(wr_return_quantity) wr_item_qty + FROM web_returns wr + INNER JOIN dateFilter d + ON wr.wr_returned_date_sk = d_date_sk + GROUP BY wr_item_sk --across all store and web channels + HAVING SUM(wr_return_quantity) > 0 + ), extract_sentiment AS + ( + SELECT pr.pr_item_sk, pr.pr_review_content, pr.pr_review_sk + FROM product_reviews pr + INNER JOIN fsr + ON pr.pr_item_sk = fsr.sr_item_sk + INNER JOIN fwr + ON fsr.sr_item_sk = fwr.wr_item_sk + WHERE pr.pr_review_content IS NOT NULL ---- add as rapids + AND abs( CAST((sr_item_qty-wr_item_qty) AS DOUBLE) / + ((sr_item_qty + wr_item_qty)/2) ) <= 0.1 + ) + SELECT * FROM extract_sentiment + ORDER BY pr_item_sk, pr_review_content, pr_review_sk + """ + merged_df = bc.sql(query) + + # second step -- Sentiment Word Extraction + merged_df["pr_review_sk"] = merged_df["pr_review_sk"].astype("int32") + merged_df["pr_review_content"] = merged_df.pr_review_content.str.lower() + merged_df["pr_review_content"] = merged_df.pr_review_content.str.replace( + [".", "?", "!"], [eol_char], regex=False + ) + + sentences = merged_df.map_partitions(create_sentences_from_reviews) + # need the global position in the sentence tokenized df + sentences["x"] = 1 + sentences['sentence_tokenized_global_pos'] = sentences['x'].cumsum() + del sentences["x"] + + word_df = sentences.map_partitions( + create_words_from_sentences, + global_position_column="sentence_tokenized_global_pos", + ) + + # This txt file comes from the official TPCx-BB kit + # We extracted it from bigbenchqueriesmr.jar + # Need to pass the absolute path for this txt file + sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") + ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) + bc.create_table('negative_sentiment', ns_df) + + sentences = sentences.persist() + wait(sentences) + bc.create_table('sentences_df', sentences) + + word_df = word_df.persist() + wait(word_df) + bc.create_table('word_df', word_df) + + merged_df = merged_df.persist() + wait(merged_df) + bc.create_table('merged_df', merged_df) + + query = """ + WITH negativesent AS + ( + SELECT distinct sentiment_word + FROM sent_df + ), word_sentence_sentiment AS + ( + SELECT sd.sentiment_word, + wd.sentence_idx_global_pos + FROM word_df wd + INNER JOIN negativesent sd ON wd.word = sd.sentiment_word + ), temp AS + ( + SELECT s.review_idx_global_pos, + w.sentiment_word, + s.sentence + FROM word_sentence_sentiment w + LEFT JOIN sentences_df s + ON w.sentence_idx_global_pos = s.sentence_tokenized_global_pos + ) + SELECT pr_item_sk AS item_sk, + sentence AS review_sentence, + 'NEG' AS sentiment, + sentiment_word + FROM temp + INNER JOIN merged_df ON pr_review_sk = review_idx_global_pos + ORDER BY pr_item_sk, review_sentence, sentiment_word + """ + result = bc.sql(query) + + bc.drop_table("sentences_df") + del sentences + bc.drop_table("word_df") + del word_df + bc.drop_table("merged_df") + del merged_df + + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py new file mode 100755 index 00000000..503dca03 --- /dev/null +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -0,0 +1,168 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster +from dask import delayed +from dask.distributed import wait +import numpy as np + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, + train_clustering_model +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +# q20 parameters +N_CLUSTERS = 8 +CLUSTER_ITERATIONS = 20 +N_ITER = 5 + + +def get_clusters(client, ml_input_df, feature_cols): + """ + Takes the dask client, kmeans_input_df and feature columns. + Returns a dictionary matching the output required for q20 + """ + import dask_cudf + ml_tasks = [ + delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) + for df in ml_input_df[feature_cols].to_delayed() + ] + + results_dict = client.compute(*ml_tasks, sync=True) + + labels = results_dict["cid_labels"] + + labels_final = dask_cudf.from_cudf(labels, npartitions=ml_input_df.npartitions) + ml_input_df["label"] = labels_final.reset_index()[0] + + output = ml_input_df[["user_sk", "label"]] + + results_dict["cid_labels"] = output + return results_dict + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_cols = [ + "ss_customer_sk", + "ss_ticket_number", + "ss_item_sk", + "ss_net_paid", + ] + store_returns_cols = [ + "sr_item_sk", + "sr_customer_sk", + "sr_ticket_number", + "sr_return_amt", + ] + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + store_returns_df = table_reader.read( + "store_returns", relevant_cols=store_returns_cols + ) + + bc.create_table("store_sales", store_sales_df) + bc.create_table("store_returns", store_returns_df) + + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("store_returns", os.path.join(data_dir, "store_returns/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = """ + SELECT + ss_customer_sk AS user_sk, + round(CASE WHEN ((returns_count IS NULL) OR (orders_count IS NULL) + OR ((returns_count / orders_count) IS NULL) ) THEN 0.0 + ELSE (returns_count / orders_count) END, 7) AS orderRatio, + round(CASE WHEN ((returns_items IS NULL) OR (orders_items IS NULL) + OR ((returns_items / orders_items) IS NULL) ) THEN 0.0 + ELSE (returns_items / orders_items) END, 7) AS itemsRatio, + round(CASE WHEN ((returns_money IS NULL) OR (orders_money IS NULL) + OR ((returns_money / orders_money) IS NULL) ) THEN 0.0 + ELSE (returns_money / orders_money) END, 7) AS monetaryRatio, + round(CASE WHEN ( returns_count IS NULL) THEN 0.0 + ELSE returns_count END, 0) AS frequency + FROM + ( + SELECT + ss_customer_sk, + -- return order ratio + CAST (COUNT(distinct(ss_ticket_number)) AS DOUBLE) + AS orders_count, + -- return ss_item_sk ratio + CAST (COUNT(ss_item_sk) AS DOUBLE) AS orders_items, + -- return monetary amount ratio + CAST(SUM( ss_net_paid ) AS DOUBLE) AS orders_money + FROM store_sales s + GROUP BY ss_customer_sk + ) orders + LEFT OUTER JOIN + ( + SELECT + sr_customer_sk, + -- return order ratio + CAST(count(distinct(sr_ticket_number)) AS DOUBLE) + AS returns_count, + -- return ss_item_sk ratio + CAST (COUNT(sr_item_sk) AS DOUBLE) AS returns_items, + -- return monetary amount ratio + CAST( SUM( sr_return_amt ) AS DOUBLE) AS returns_money + FROM store_returns + GROUP BY sr_customer_sk + ) returned ON ss_customer_sk=sr_customer_sk + """ + final_df = bc.sql(query) + + final_df = final_df.fillna(0) + final_df = final_df.repartition(npartitions=1).persist() + wait(final_df) + + final_df = final_df.sort_values(["user_sk"]).reset_index(drop=True) + final_df = final_df.persist() + wait(final_df) + + feature_cols = ["orderRatio", "itemsRatio", "monetaryRatio", "frequency"] + + results_dict = get_clusters( + client=client, ml_input_df=final_df, feature_cols=feature_cols + ) + + return results_dict + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py new file mode 100755 index 00000000..0ed1fffe --- /dev/null +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -0,0 +1,173 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +store_sales_cols = [ + "ss_item_sk", + "ss_store_sk", + "ss_customer_sk", + "ss_ticket_number", + "ss_quantity", + "ss_sold_date_sk", +] +date_cols = ["d_date_sk", "d_year", "d_moy"] +websale_cols = ["ws_item_sk", "ws_bill_customer_sk", "ws_quantity", "ws_sold_date_sk"] +sr_cols = [ + "sr_item_sk", + "sr_customer_sk", + "sr_ticket_number", + "sr_return_quantity", + "sr_returned_date_sk", +] +store_cols = ["s_store_name", "s_store_id", "s_store_sk"] +item_cols = ["i_item_id", "i_item_desc", "i_item_sk"] + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) + store_returns_df = table_reader.read("store_returns", relevant_cols=sr_cols) + store_table_df = table_reader.read("store", relevant_cols=store_cols) + item_table_df = table_reader.read("item", relevant_cols=item_cols) + + bc.create_table("store_sales", store_sales_df) + bc.create_table("date_dim", date_dim_df) + bc.create_table("item", item_table_df) + bc.create_table("web_sales", web_sales_df) + bc.create_table("store_returns", store_returns_df) + bc.create_table("store", store_table_df) + + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table("store_returns", os.path.join(data_dir, "store_returns/*.parquet")) + # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = """ + SELECT + part_i.i_item_id AS i_item_id, + part_i.i_item_desc AS i_item_desc, + part_s.s_store_id AS s_store_id, + part_s.s_store_name AS s_store_name, + CAST(SUM(part_ss.ss_quantity) AS BIGINT) AS store_sales_quantity, + CAST(SUM(part_sr.sr_return_quantity) AS BIGINT) AS store_returns_quantity, + CAST(SUM(part_ws.ws_quantity) AS BIGINT) AS web_sales_quantity + FROM + ( + SELECT + sr_item_sk, + sr_customer_sk, + sr_ticket_number, + sr_return_quantity + FROM + store_returns sr, + date_dim d2 + WHERE d2.d_year = 2003 + AND d2.d_moy BETWEEN 1 AND 7 --which were returned in the next six months + AND sr.sr_returned_date_sk = d2.d_date_sk + ) part_sr + INNER JOIN + ( + SELECT + ws_item_sk, + ws_bill_customer_sk, + ws_quantity + FROM + web_sales ws, + date_dim d3 + -- in the following three years (re-purchased by the returning customer afterwards through the web sales channel) + WHERE d3.d_year BETWEEN 2003 AND 2005 + AND ws.ws_sold_date_sk = d3.d_date_sk + ) part_ws ON + ( + part_sr.sr_item_sk = part_ws.ws_item_sk + AND part_sr.sr_customer_sk = part_ws.ws_bill_customer_sk + ) INNER JOIN + ( + SELECT + ss_item_sk, + ss_store_sk, + ss_customer_sk, + ss_ticket_number, + ss_quantity + FROM + store_sales ss, + date_dim d1 + WHERE d1.d_year = 2003 + AND d1.d_moy = 1 + AND ss.ss_sold_date_sk = d1.d_date_sk + ) part_ss ON + ( + part_ss.ss_ticket_number = part_sr.sr_ticket_number + AND part_ss.ss_item_sk = part_sr.sr_item_sk + AND part_ss.ss_customer_sk = part_sr.sr_customer_sk + ) + INNER JOIN store part_s ON + ( + part_s.s_store_sk = part_ss.ss_store_sk + ) + INNER JOIN item part_i ON + ( + part_i.i_item_sk = part_ss.ss_item_sk + ) + GROUP BY + part_i.i_item_id, + part_i.i_item_desc, + part_s.s_store_id, + part_s.s_store_name + ORDER BY + part_i.i_item_id, + part_i.i_item_desc, + part_s.s_store_id, + part_s.s_store_name + LIMIT 100 + """ + result = bc.sql(query) + result['i_item_desc'] = result['i_item_desc'].str.strip() + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py new file mode 100755 index 00000000..6ba0e5b0 --- /dev/null +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -0,0 +1,123 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +# -------- Q22 ----------- +q22_date = "2001-05-08" +q22_i_current_price_min = "0.98" +q22_i_current_price_max = "1.5" + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + inv_columns = [ + "inv_item_sk", + "inv_warehouse_sk", + "inv_date_sk", + "inv_quantity_on_hand", + ] + inventory = table_reader.read("inventory", relevant_cols=inv_columns) + + item_columns = ["i_item_id", "i_current_price", "i_item_sk"] + item = table_reader.read("item", relevant_cols=item_columns) + + warehouse_columns = ["w_warehouse_sk", "w_warehouse_name"] + warehouse = table_reader.read("warehouse", relevant_cols=warehouse_columns) + + dd_columns = ["d_date_sk", "d_date"] + date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) + + bc.create_table('inventory', inventory) + bc.create_table('item', item) + bc.create_table('warehouse', warehouse) + bc.create_table('date_dim', date_dim) + + # bc.create_table('inventory', os.path.join(data_dir, "inventory/*.parquet")) + # bc.create_table('item', os.path.join(data_dir, "item/*.parquet")) + # bc.create_table('warehouse', os.path.join(data_dir, "warehouse/*.parquet")) + # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + SELECT + w_warehouse_name, + i_item_id, + SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) + / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) AS inv_before, + SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) + / 1000000 >= 0 THEN inv_quantity_on_hand ELSE 0 END) AS inv_after + FROM + inventory inv, + item i, + warehouse w, + date_dim d + WHERE i_current_price BETWEEN {q22_i_current_price_min} AND {q22_i_current_price_max} + AND i_item_sk = inv_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 >= -30 + AND timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 <= 30 + GROUP BY w_warehouse_name, i_item_id + HAVING SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date}', CAST(d_date || ' 00:00:00' AS timestamp)) + / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) > 0 + AND + ( + CAST( + SUM (CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 >= 0 THEN inv_quantity_on_hand ELSE 0 END) AS DOUBLE) + / CAST( SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) + AS DOUBLE) >= 0.666667 + ) + AND + ( + CAST( + SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 >= 0 THEN inv_quantity_on_hand ELSE 0 END) AS DOUBLE) + / CAST ( SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) + AS DOUBLE) <= 1.50 + ) + ORDER BY w_warehouse_name, i_item_id + LIMIT 100 + """ + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py new file mode 100755 index 00000000..610ca385 --- /dev/null +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -0,0 +1,160 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q23 ----------- +q23_year = 2001 +q23_month = 1 +q23_coefficient = 1.3 + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + + date_cols = ["d_date_sk", "d_year", "d_moy"] + date_df = table_reader.read("date_dim", relevant_cols=date_cols) + + inv_cols = [ + "inv_warehouse_sk", + "inv_item_sk", + "inv_date_sk", + "inv_quantity_on_hand", + ] + inv_df = table_reader.read("inventory", relevant_cols=inv_cols) + + bc.create_table('inventory', inv_df) + bc.create_table('date_dim', date_df) + + # bc.create_table('inventory', os.path.join(data_dir, "inventory/*.parquet")) + # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = f""" + SELECT inv_warehouse_sk, + inv_item_sk, + inv_quantity_on_hand, + d_moy + FROM inventory inv + INNER JOIN date_dim d ON inv.inv_date_sk = d.d_date_sk + AND d.d_year = {q23_year} + AND d_moy between {q23_month} AND {q23_month + 1} + """ + inv_dates_result = bc.sql(query_1) + + inv_dates_result = inv_dates_result.persist() + wait(inv_dates_result) + bc.create_table('inv_dates', inv_dates_result) + query_2 = """ + SELECT inv_warehouse_sk, + inv_item_sk, + d_moy, + AVG(CAST(inv_quantity_on_hand AS DOUBLE)) AS q_mean + FROM inv_dates + GROUP BY inv_warehouse_sk, inv_item_sk, d_moy + """ + mean_result = bc.sql(query_2) + + mean_result = mean_result.persist() + wait(mean_result) + bc.create_table('mean_df', mean_result) + query_3 = """ + SELECT id.inv_warehouse_sk, + id.inv_item_sk, + id.d_moy, + md.q_mean, + SQRT( SUM( (id.inv_quantity_on_hand - md.q_mean) * (id.inv_quantity_on_hand - md.q_mean) ) + / (COUNT(id.inv_quantity_on_hand) - 1.0)) AS q_std + FROM mean_df md + INNER JOIN inv_dates id ON id.inv_warehouse_sk = md.inv_warehouse_sk + AND id.inv_item_sk = md.inv_item_sk + AND id.d_moy = md.d_moy + AND md.q_mean > 0.0 + GROUP BY id.inv_warehouse_sk, id.inv_item_sk, id.d_moy, md.q_mean + """ + std_result = bc.sql(query_3) + + bc.drop_table("inv_dates") + del inv_dates_result + + bc.drop_table("mean_df") + del mean_result + + std_result = std_result.persist() + wait(std_result) + bc.create_table('iteration', std_result) + query_4 = f""" + SELECT inv_warehouse_sk, + inv_item_sk, + d_moy, + q_std / q_mean AS qty_cov + FROM iteration + WHERE (q_std / q_mean) >= {q23_coefficient} + """ + std_result = bc.sql(query_4) + + bc.drop_table("iteration") + + std_result = std_result.persist() + wait(std_result) + bc.create_table('temp_table', std_result) + query = f""" + SELECT inv1.inv_warehouse_sk, + inv1.inv_item_sk, + inv1.d_moy, + inv1.qty_cov AS cov, + inv2.d_moy AS inv2_d_moy, + inv2.qty_cov AS inv2_cov + FROM temp_table inv1 + INNER JOIN temp_table inv2 ON inv1.inv_warehouse_sk = inv2.inv_warehouse_sk + AND inv1.inv_item_sk = inv2.inv_item_sk + AND inv1.d_moy = {q23_month} + AND inv2.d_moy = {q23_month + 1} + ORDER BY inv1.inv_warehouse_sk, + inv1.inv_item_sk + """ + result = bc.sql(query) + + bc.drop_table("temp_table") + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py new file mode 100755 index 00000000..1c7802e9 --- /dev/null +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -0,0 +1,123 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + +ws_cols = ["ws_item_sk", "ws_sold_date_sk", "ws_quantity"] +item_cols = ["i_item_sk", "i_current_price"] +imp_cols = [ + "imp_item_sk", + "imp_competitor_price", + "imp_start_date", + "imp_end_date", + "imp_sk", +] +ss_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_quantity"] + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + ### read tables + ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + imp_df = table_reader.read("item_marketprices", relevant_cols=imp_cols) + ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) + + bc.create_table("web_sales", ws_df) + bc.create_table("item", item_df) + bc.create_table("item_marketprices", imp_df) + bc.create_table("store_sales", ss_df) + + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + # bc.create_table("item_marketprices", os.path.join(data_dir, "item_marketprices/*.parquet")) + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = """ + WITH temp_table as + ( + SELECT + i_item_sk, + imp_sk, + (imp_competitor_price - i_current_price) / i_current_price AS price_change, + imp_start_date, + (imp_end_date - imp_start_date) AS no_days_comp_price + FROM item i ,item_marketprices imp + WHERE i.i_item_sk = imp.imp_item_sk + AND i.i_item_sk = 10000 + ORDER BY i_item_sk, imp_sk, imp_start_date + ) + SELECT ws_item_sk, + -- avg ( (current_ss_quant + current_ws_quant - prev_ss_quant - prev_ws_quant) / ((prev_ss_quant + prev_ws_quant) * ws.price_change) ) -- single node + sum( (current_ss_quant+current_ws_quant-prev_ss_quant-prev_ws_quant) / (prev_ss_quant*ws.price_change+prev_ws_quant*ws.price_change) ) + / count( (current_ss_quant + current_ws_quant - prev_ss_quant - prev_ws_quant) / ((prev_ss_quant + prev_ws_quant) * ws.price_change) ) AS cross_price_elasticity + FROM + ( + SELECT + ws_item_sk, + imp_sk, + price_change, + SUM( CASE WHEN ( (ws_sold_date_sk >= c.imp_start_date) AND (ws_sold_date_sk < (c.imp_start_date + c.no_days_comp_price))) THEN ws_quantity ELSE 0 END ) AS current_ws_quant, + SUM( CASE WHEN ( (ws_sold_date_sk >= (c.imp_start_date - c.no_days_comp_price)) AND (ws_sold_date_sk < c.imp_start_date)) THEN ws_quantity ELSE 0 END ) AS prev_ws_quant + FROM web_sales ws + JOIN temp_table c ON ws.ws_item_sk = c.i_item_sk + GROUP BY ws_item_sk, imp_sk, price_change + ) ws JOIN + ( + SELECT + ss_item_sk, + imp_sk, + price_change, + SUM( CASE WHEN ((ss_sold_date_sk >= c.imp_start_date) AND (ss_sold_date_sk < (c.imp_start_date + c.no_days_comp_price))) THEN ss_quantity ELSE 0 END) AS current_ss_quant, + SUM( CASE WHEN ((ss_sold_date_sk >= (c.imp_start_date - c.no_days_comp_price)) AND (ss_sold_date_sk < c.imp_start_date)) THEN ss_quantity ELSE 0 END) AS prev_ss_quant + FROM store_sales ss + JOIN temp_table c ON c.i_item_sk = ss.ss_item_sk + GROUP BY ss_item_sk, imp_sk, price_change + ) ss + ON (ws.ws_item_sk = ss.ss_item_sk and ws.imp_sk = ss.imp_sk) + GROUP BY ws.ws_item_sk + """ + + result = bc.sql(query) + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py new file mode 100755 index 00000000..b1d3a528 --- /dev/null +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -0,0 +1,153 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, + train_clustering_model +) +from dask import delayed + +from bdb_tools.readers import build_reader + +from dask_sql import Context + + +# -------- Q25 ----------- +# -- store_sales and web_sales date +q25_date = "2002-01-02" + +N_CLUSTERS = 8 +CLUSTER_ITERATIONS = 20 +N_ITER = 5 + + +def get_clusters(client, ml_input_df): + import dask_cudf + + ml_tasks = [ + delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) + for df in ml_input_df.to_delayed() + ] + results_dict = client.compute(*ml_tasks, sync=True) + + output = ml_input_df.index.to_frame().reset_index(drop=True) + + labels_final = dask_cudf.from_cudf( + results_dict["cid_labels"], npartitions=output.npartitions + ) + output["label"] = labels_final.reset_index()[0] + + # Based on CDH6.1 q25-result formatting + results_dict["cid_labels"] = output + return results_dict + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ss_cols = ["ss_customer_sk", "ss_sold_date_sk", "ss_ticket_number", "ss_net_paid"] + ws_cols = [ + "ws_bill_customer_sk", + "ws_sold_date_sk", + "ws_order_number", + "ws_net_paid", + ] + datedim_cols = ["d_date_sk", "d_date"] + + ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) + ws_ddf = table_reader.read("web_sales", relevant_cols=ws_cols, index=False) + datedim_ddf = table_reader.read("date_dim", relevant_cols=datedim_cols, index=False) + + bc.create_table("web_sales", ws_ddf) + bc.create_table("store_sales", ss_ddf) + bc.create_table("date_dim", datedim_ddf) + + # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + WITH concat_table AS + ( + ( + SELECT + ss_customer_sk AS cid, + count(distinct ss_ticket_number) AS frequency, + max(ss_sold_date_sk) AS most_recent_date, + CAST( SUM(ss_net_paid) AS DOUBLE) AS amount + FROM store_sales ss + JOIN date_dim d ON ss.ss_sold_date_sk = d.d_date_sk + WHERE CAST(d.d_date AS DATE) > DATE '{q25_date}' + AND ss_customer_sk IS NOT NULL + GROUP BY ss_customer_sk + ) union all + ( + SELECT + ws_bill_customer_sk AS cid, + count(distinct ws_order_number) AS frequency, + max(ws_sold_date_sk) AS most_recent_date, + CAST( SUM(ws_net_paid) AS DOUBLE) AS amount + FROM web_sales ws + JOIN date_dim d ON ws.ws_sold_date_sk = d.d_date_sk + WHERE CAST(d.d_date AS DATE) > DATE '{q25_date}' + AND ws_bill_customer_sk IS NOT NULL + GROUP BY ws_bill_customer_sk + ) + ) + SELECT + cid AS cid, + CASE WHEN 37621 - max(most_recent_date) < 60 THEN 1.0 + ELSE 0.0 END AS recency, -- 37621 == 2003-01-02 + CAST( SUM(frequency) AS BIGINT) AS frequency, --total frequency + CAST( SUM(amount) AS DOUBLE) AS amount --total amount + FROM concat_table + GROUP BY cid + ORDER BY cid + """ + cluster_input_ddf = bc.sql(query) + + # Prepare df for KMeans clustering + cluster_input_ddf["recency"] = cluster_input_ddf["recency"].astype("int64") + + cluster_input_ddf = cluster_input_ddf.repartition(npartitions=1) + cluster_input_ddf = cluster_input_ddf.persist() + cluster_input_ddf = cluster_input_ddf.set_index('cid') + results_dict = get_clusters(client=client, ml_input_df=cluster_input_ddf) + + return results_dict + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config, create_blazing_context=True) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py new file mode 100755 index 00000000..ec0c4203 --- /dev/null +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -0,0 +1,131 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, + train_clustering_model +) + +from bdb_tools.readers import build_reader + +from dask import delayed + +from dask_sql import Context + +# -------- Q26 ----------- +q26_i_category_IN = "Books" +q26_count_ss_item_sk = 5 + +N_CLUSTERS = 8 +CLUSTER_ITERATIONS = 20 +N_ITER = 5 + + +def get_clusters(client, kmeans_input_df): + import dask_cudf + + ml_tasks = [ + delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) + for df in kmeans_input_df.to_delayed() + ] + + results_dict = client.compute(*ml_tasks, sync=True) + + output = kmeans_input_df.index.to_frame().reset_index(drop=True) + + labels_final = dask_cudf.from_cudf( + results_dict["cid_labels"], npartitions=output.npartitions + ) + output["label"] = labels_final.reset_index()[0] + + # Based on CDH6.1 q26-result formatting + results_dict["cid_labels"] = output + return results_dict + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ss_cols = ["ss_customer_sk", "ss_item_sk"] + items_cols = ["i_item_sk", "i_category", "i_class_id"] + + ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) + items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) + + bc.create_table("store_sales", ss_ddf) + bc.create_table("item", items_ddf) + + # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) + # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query = f""" + SELECT + ss.ss_customer_sk AS cid, + CAST( count(CASE WHEN i.i_class_id=1 THEN 1 ELSE NULL END) AS DOUBLE ) AS id1, + CAST( count(CASE WHEN i.i_class_id=2 THEN 1 ELSE NULL END) AS DOUBLE ) AS id2, + CAST( count(CASE WHEN i.i_class_id=3 THEN 1 ELSE NULL END) AS DOUBLE ) AS id3, + CAST( count(CASE WHEN i.i_class_id=4 THEN 1 ELSE NULL END) AS DOUBLE ) AS id4, + CAST( count(CASE WHEN i.i_class_id=5 THEN 1 ELSE NULL END) AS DOUBLE ) AS id5, + CAST( count(CASE WHEN i.i_class_id=6 THEN 1 ELSE NULL END) AS DOUBLE ) AS id6, + CAST( count(CASE WHEN i.i_class_id=7 THEN 1 ELSE NULL END) AS DOUBLE ) AS id7, + CAST( count(CASE WHEN i.i_class_id=8 THEN 1 ELSE NULL END) AS DOUBLE ) AS id8, + CAST( count(CASE WHEN i.i_class_id=9 THEN 1 ELSE NULL END) AS DOUBLE ) AS id9, + CAST( count(CASE WHEN i.i_class_id=10 THEN 1 ELSE NULL END) AS DOUBLE ) AS id10, + CAST( count(CASE WHEN i.i_class_id=11 THEN 1 ELSE NULL END) AS DOUBLE ) AS id11, + CAST( count(CASE WHEN i.i_class_id=12 THEN 1 ELSE NULL END) AS DOUBLE ) AS id12, + CAST( count(CASE WHEN i.i_class_id=13 THEN 1 ELSE NULL END) AS DOUBLE ) AS id13, + CAST( count(CASE WHEN i.i_class_id=14 THEN 1 ELSE NULL END) AS DOUBLE ) AS id14, + CAST( count(CASE WHEN i.i_class_id=15 THEN 1 ELSE NULL END) AS DOUBLE ) AS id15 + FROM store_sales ss + INNER JOIN item i + ON + ( + ss.ss_item_sk = i.i_item_sk + AND i.i_category IN ('{q26_i_category_IN}') + AND ss.ss_customer_sk IS NOT NULL + ) + GROUP BY ss.ss_customer_sk + HAVING count(ss.ss_item_sk) > {q26_count_ss_item_sk} + ORDER BY cid + """ + result = bc.sql(query) + result = result.repartition(npartitions=1) + result_ml = result.set_index('cid') + ml_result_dict = get_clusters(client=client, kmeans_input_df=result_ml) + return ml_result_dict + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py new file mode 100755 index 00000000..9d4ec599 --- /dev/null +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -0,0 +1,152 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.text import ( + create_sentences_from_reviews, + create_words_from_sentences +) + +from bdb_tools.cluster_startup import attach_to_cluster +from dask.distributed import wait +import spacy + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q27 ----------- +q27_pr_item_sk = 10002 +EOL_CHAR = "." + + +def read_tables(data_dir, bc): + ### splitting by row groups for better parallelism + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] + product_reviews_df = table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols + ) + + bc.create_table("product_reviews", product_reviews_df) + + # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) + + +def ner_parser(df, col_string, batch_size=256): + spacy.require_gpu() + nlp = spacy.load("en_core_web_sm") + docs = nlp.pipe(df[col_string], disable=["tagger", "parser"], batch_size=batch_size) + out = [] + for doc in docs: + l = [ent.text for ent in doc.ents if ent.label_ == "ORG"] + val = ", " + l = val.join(l) + out.append(l) + df["company_name_list"] = out + return df + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + import dask_cudf + + query = f""" + SELECT pr_review_sk, pr_item_sk, pr_review_content + FROM product_reviews + WHERE pr_item_sk = {q27_pr_item_sk} + """ + product_reviews_df = bc.sql(query) + + sentences = product_reviews_df.map_partitions( + create_sentences_from_reviews, + review_column="pr_review_content", + end_of_line_char=EOL_CHAR, + ) + + # need the global position in the sentence tokenized df + sentences["x"] = 1 + sentences["sentence_tokenized_global_pos"] = sentences.x.cumsum() + del sentences["x"] + del product_reviews_df + + # Do the NER + sentences = sentences.to_dask_dataframe() + ner_parsed = sentences.map_partitions(ner_parser, "sentence") + ner_parsed = dask_cudf.from_dask_dataframe(ner_parsed) + ner_parsed = ner_parsed.persist() + wait(ner_parsed) + + ner_parsed = ner_parsed[ner_parsed.company_name_list != ""] + + # separate NER results into one row per found company + repeated_names = ner_parsed.map_partitions( + create_words_from_sentences, + sentence_column="company_name_list", + global_position_column="sentence_tokenized_global_pos", + delimiter="รฉ", + ) + del sentences + + # recombine + repeated_names = repeated_names.persist() + wait(repeated_names) + bc.create_table('repeated_names', repeated_names) + + ner_parsed = ner_parsed.persist() + wait(ner_parsed) + bc.create_table('ner_parsed', ner_parsed) + + query = f""" + SELECT review_idx_global_pos as review_sk, + CAST({q27_pr_item_sk} AS BIGINT) as item_sk, + word as company_name, + sentence as review_sentence + FROM repeated_names left join ner_parsed + ON sentence_idx_global_pos = sentence_tokenized_global_pos + ORDER BY review_idx_global_pos, item_sk, word, sentence + """ + recombined = bc.sql(query) + + bc.drop_table("repeated_names") + bc.drop_table("ner_parsed") + del ner_parsed + del repeated_names + return recombined + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) + diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py new file mode 100755 index 00000000..16b8a28d --- /dev/null +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -0,0 +1,372 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os +import time + +from bdb_tools.cluster_startup import attach_to_cluster +from cuml.feature_extraction.text import HashingVectorizer +import cupy +import dask +from distributed import wait +import cupy as cp +import numpy as np + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask_sql import Context + + +N_FEATURES = 2 ** 23 # Spark is doing 2^20 +ngram_range = (1, 2) +preprocessor = lambda s:s.str.lower() +norm = None +alternate_sign = False + + +def gpu_hashing_vectorizer(x): + vec = HashingVectorizer(n_features=N_FEATURES, + alternate_sign=alternate_sign, + ngram_range=ngram_range, + norm=norm, + preprocessor=preprocessor + ) + return vec.fit_transform(x) + + +def map_labels(ser): + import cudf + output_ser = cudf.Series(cudf.core.column.full(size=len(ser), fill_value=2, dtype=np.int32)) + zero_flag = (ser==1) | (ser==2) + output_ser.loc[zero_flag]=0 + + three_flag = (ser==3) + output_ser.loc[three_flag]=1 + + return output_ser + + +def build_features(t): + X = t["pr_review_content"] + X = X.map_partitions( + gpu_hashing_vectorizer, + meta=dask.array.from_array( + cupy.sparse.csr_matrix(cupy.zeros(1, dtype=cp.float32)) + ), + ) + + X = X.astype(np.float32).persist() + X.compute_chunk_sizes() + + return X + + +def build_labels(reviews_df): + y = reviews_df["pr_review_rating"].map_partitions(map_labels) + y = y.map_partitions(lambda x: cupy.asarray(x, cupy.int32)).persist() + y.compute_chunk_sizes() + + return y + + +def categoricalize(num_sr): + return num_sr.astype("str").str.replace(["0", "1", "2"], ["NEG", "NEUT", "POS"]) + + +def sum_tp_fp(y_y_pred, nclasses): + + y, y_pred = y_y_pred + res = cp.zeros((nclasses, 2), order="F") + + for i in range(nclasses): + pos_pred_ix = cp.where(y_pred == i)[0] + + # short circuit + if len(pos_pred_ix) == 0: + res[i] = 0 + break + + tp_sum = (y_pred[pos_pred_ix] == y[pos_pred_ix]).sum() + fp_sum = (y_pred[pos_pred_ix] != y[pos_pred_ix]).sum() + res[i][0] = tp_sum + res[i][1] = fp_sum + return res + + +def precision_score(client, y, y_pred, average="binary"): + from cuml.dask.common.input_utils import DistributedDataHandler + + nclasses = len(cp.unique(y.map_blocks(lambda x: cp.unique(x)).compute())) + + if average == "binary" and nclasses > 2: + raise ValueError + + if nclasses < 2: + raise ValueError("Single class precision is not yet supported") + + ddh = DistributedDataHandler.create([y, y_pred]) + + precision_scores = client.compute( + [ + client.submit(sum_tp_fp, part, nclasses, workers=[worker]) + for worker, part in ddh.gpu_futures + ], + sync=True, + ) + + res = cp.zeros((nclasses, 2), order="F") + + for i in precision_scores: + res += i + + if average == "binary" or average == "macro": + + prec = cp.zeros(nclasses) + for i in range(nclasses): + tp_sum, fp_sum = res[i] + prec[i] = (tp_sum / (tp_sum + fp_sum)).item() + + if average == "binary": + return prec[nclasses - 1].item() + else: + return prec.mean().item() + else: + global_tp = cp.sum(res[:, 0]) + global_fp = cp.sum(res[:, 1]) + + return global_tp / (global_tp + global_fp).item() + + +def local_cm(y_y_pred, unique_labels, sample_weight): + + y_true, y_pred = y_y_pred + labels = unique_labels + + n_labels = labels.size + + # Assume labels are monotonically increasing for now. + + # intersect y_pred, y_true with labels, eliminate items not in labels + ind = cp.logical_and(y_pred < n_labels, y_true < n_labels) + y_pred = y_pred[ind] + y_true = y_true[ind] + + if sample_weight is None: + sample_weight = cp.ones(y_true.shape[0], dtype=np.int64) + else: + sample_weight = cp.asarray(sample_weight) + + sample_weight = sample_weight[ind] + + cm = cp.sparse.coo_matrix( + (sample_weight, (y_true, y_pred)), shape=(n_labels, n_labels), dtype=cp.float32, + ).toarray() + + return cp.nan_to_num(cm) + + +def confusion_matrix(client, y_true, y_pred, normalize=None, sample_weight=None): + from cuml.dask.common.input_utils import DistributedDataHandler + + unique_classes = cp.unique(y_true.map_blocks(lambda x: cp.unique(x)).compute()) + nclasses = len(unique_classes) + + ddh = DistributedDataHandler.create([y_true, y_pred]) + + cms = client.compute( + [ + client.submit( + local_cm, part, unique_classes, sample_weight, workers=[worker] + ) + for worker, part in ddh.gpu_futures + ], + sync=True, + ) + + cm = cp.zeros((nclasses, nclasses)) + for i in cms: + cm += i + + with np.errstate(all="ignore"): + if normalize == "true": + cm = cm / cm.sum(axis=1, keepdims=True) + elif normalize == "pred": + cm = cm / cm.sum(axis=0, keepdims=True) + elif normalize == "all": + cm = cm / cm.sum() + cm = cp.nan_to_num(cm) + + return cm + + +def accuracy_score(client, y, y_hat): + from uuid import uuid1 + from cuml.dask.common.input_utils import DistributedDataHandler + + ddh = DistributedDataHandler.create([y_hat, y]) + + def _count_accurate_predictions(y_hat_y): + y_hat, y = y_hat_y + y_hat = cp.asarray(y_hat, dtype=y_hat.dtype) + y = cp.asarray(y, dtype=y.dtype) + return y.shape[0] - cp.count_nonzero(y - y_hat) + + key = uuid1() + + futures = client.compute( + [ + client.submit( + _count_accurate_predictions, + worker_future[1], + workers=[worker_future[0]], + key="%s-%s" % (key, idx), + ) + for idx, worker_future in enumerate(ddh.gpu_futures) + ], + sync=True, + ) + + return sum(futures) / y.shape[0] + + +def post_etl_processing(client, train_data, test_data): + import cudf + from cuml.dask.naive_bayes import MultinomialNB as DistMNB + from cuml.dask.common import to_dask_cudf + from cuml.dask.common.input_utils import DistributedDataHandler + + # Feature engineering + X_train = build_features(train_data) + X_test = build_features(test_data) + + y_train = build_labels(train_data) + y_test = build_labels(test_data) + + # Perform ML + model = DistMNB(client=client, alpha=0.001) + model.fit(X_train, y_train) + + ### this regression seems to be coming from here + test_pred_st = time.time() + y_hat = model.predict(X_test).persist() + + # Compute distributed performance metrics + acc = accuracy_score(client, y_test, y_hat) + + print("Accuracy: " + str(acc)) + prec = precision_score(client, y_test, y_hat, average="macro") + + print("Precision: " + str(prec)) + cmat = confusion_matrix(client, y_test, y_hat) + + print("Confusion Matrix: " + str(cmat)) + metric_et = time.time() + + # Place results back in original Dataframe + + ddh = DistributedDataHandler.create(y_hat) + test_preds = to_dask_cudf( + [client.submit(cudf.Series, part) for w, part in ddh.gpu_futures] + ) + + test_preds = test_preds.map_partitions(categoricalize) + + test_data["prediction"] = test_preds + + final_data = test_data[["pr_review_sk", "pr_review_rating", "prediction"]].persist() + + final_data = final_data.sort_values("pr_review_sk").reset_index(drop=True) + wait(final_data) + return final_data, acc, prec, cmat + + +def read_tables(data_dir, bc): + ### splitting by row groups for better parallelism + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + + columns = [ + "pr_review_content", + "pr_review_rating", + "pr_review_sk", + ] + pr_df = table_reader.read("product_reviews", relevant_cols=columns) + + bc.create_table("product_reviews", pr_df) + + # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + # 10 % of data + query1 = """ + SELECT + pr_review_sk, + pr_review_rating, + pr_review_content + FROM product_reviews + WHERE mod(pr_review_sk, 10) IN (0) + AND pr_review_content IS NOT NULL + ORDER BY pr_review_sk + """ + test_data = bc.sql(query1) + + # 90 % of data + query2 = """ + SELECT + pr_review_sk, + pr_review_rating, + pr_review_content + FROM product_reviews + WHERE mod(pr_review_sk, 10) IN (1,2,3,4,5,6,7,8,9) + AND pr_review_content IS NOT NULL + ORDER BY pr_review_sk + """ + train_data = bc.sql(query2) + + final_data, acc, prec, cmat = post_etl_processing( + client=client, train_data=train_data, test_data=test_data + ) + + payload = { + "df": final_data, + "acc": acc, + "prec": prec, + "cmat": cmat, + "output_type": "supervised", + } + + return payload + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py new file mode 100755 index 00000000..8015916a --- /dev/null +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -0,0 +1,93 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q29 ----------- +q29_limit = 100 + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + item_cols = ["i_item_sk", "i_category_id"] + item_df = table_reader.read("item", relevant_cols=item_cols) + + ws_cols = ["ws_order_number", "ws_item_sk"] + ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) + + bc.create_table('item', item_df) + bc.create_table('web_sales', ws_df) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_distinct = """ + SELECT DISTINCT i_category_id, ws_order_number + FROM web_sales ws, item i + WHERE ws.ws_item_sk = i.i_item_sk + AND i.i_category_id IS NOT NULL + """ + result_distinct = bc.sql(query_distinct) + + result_distinct = result_distinct.persist() + wait(result_distinct) + bc.create_table('distinct_table', result_distinct) + + query = f""" + SELECT category_id_1, category_id_2, COUNT (*) AS cnt + FROM + ( + SELECT CAST(t1.i_category_id as BIGINT) AS category_id_1, + CAST(t2.i_category_id as BIGINT) AS category_id_2 + FROM distinct_table t1 + INNER JOIN distinct_table t2 + ON t1.ws_order_number = t2.ws_order_number + WHERE t1.i_category_id < t2.i_category_id + ) + GROUP BY category_id_1, category_id_2 + ORDER BY cnt DESC, category_id_1, category_id_2 + LIMIT {q29_limit} + """ + result = bc.sql(query) + + bc.drop_table("distinct_table") + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py new file mode 100755 index 00000000..2a0464ac --- /dev/null +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -0,0 +1,131 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2020, BlazingSQL, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os + +from bdb_tools.cluster_startup import attach_to_cluster + +from bdb_tools.utils import ( + benchmark, + gpubdb_argparser, + run_query, +) + +from bdb_tools.sessionization import ( + get_distinct_sessions, + get_pairs +) + +from bdb_tools.readers import build_reader + +from dask.distributed import wait + +from dask_sql import Context + +# -------- Q30 ----------- +# session timeout in secs +q30_session_timeout_inSec = 3600 +# query output limit +q30_limit = 40 + + +def read_tables(data_dir, bc): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_cols = ["i_category_id", "i_item_sk"] + item_df = table_reader.read("item", relevant_cols=item_cols) + + wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + bc.create_table('web_clickstreams', wcs_df) + bc.create_table('item', item_df) + + # bc.create_table('web_clickstreams', os.path.join(data_dir, "web_clickstreams/*.parquet")) + # bc.create_table('item', os.path.join(data_dir, "item/*.parquet")) + + +def main(data_dir, client, bc, config): + benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + + query_1 = """ + SELECT i_item_sk, + CAST(i_category_id AS TINYINT) AS i_category_id + FROM item + """ + item_df = bc.sql(query_1) + + item_df = item_df.persist() + wait(item_df) + bc.create_table("item_df", item_df) + + query_2 = """ + SELECT wcs_user_sk, + (wcs_click_date_sk * 86400 + wcs_click_time_sk) AS tstamp_inSec, + i_category_id + FROM web_clickstreams wcs, item_df i + WHERE wcs.wcs_item_sk = i.i_item_sk + AND i.i_category_id IS NOT NULL + AND wcs.wcs_user_sk IS NOT NULL + ORDER BY wcs.wcs_user_sk, tstamp_inSec, i_category_id + """ + merged_df = bc.sql(query_2) + + bc.drop_table("item_df") + del item_df + + distinct_session_df = merged_df.map_partitions(get_distinct_sessions, + keep_cols=["wcs_user_sk", "i_category_id"], + time_out=q30_session_timeout_inSec) + + del merged_df + pair_df = distinct_session_df.map_partitions( + get_pairs, + pair_col="i_category_id", + output_col_1="category_id_1", + output_col_2="category_id_2") + del distinct_session_df + + pair_df = pair_df.persist() + wait(pair_df) + bc.create_table('pair_df', pair_df) + + last_query = f""" + SELECT CAST(category_id_1 AS BIGINT) AS category_id_1, + CAST(category_id_2 AS BIGINT) AS category_id_2, + COUNT(category_id_2) AS cnt + FROM pair_df + GROUP BY category_id_1, category_id_2 + ORDER BY cnt desc + LIMIT {q30_limit} + """ + result = bc.sql(last_query) + + bc.drop_table("pair_df") + return result + + +if __name__ == "__main__": + config = gpubdb_argparser() + client, _ = attach_to_cluster(config) + c = Context() + run_query(config=config, client=client, query_func=main, blazing_context=c) From b14335ad33b887b160dd14f2b9b68b59d5dc60bf Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Thu, 12 Aug 2021 15:28:00 -0700 Subject: [PATCH 02/51] Fixes --- gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py | 4 +++- gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py | 4 +++- gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index b0c92ec9..e6700d44 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -22,6 +22,8 @@ import numpy as np import cupy as cp +import dask_cudf + from bdb_tools.text import create_sentences_from_reviews, create_words_from_sentences from bdb_tools.utils import ( @@ -292,7 +294,7 @@ def main(data_dir, client, bc, config): # Need to pass the absolute path for this txt file sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) - bc.create_table('negative_sentiment', ns_df) + bc.create_table('sent_df', ns_df) word_df = word_df.persist() wait(word_df) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index 4e6c2b26..661552e5 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -18,6 +18,8 @@ import sys import os +import dask_cudf + from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( @@ -151,7 +153,7 @@ def main(data_dir, client, bc, config): # Need to pass the absolute path for this txt file sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) - bc.create_table('negative_sentiment', ns_df) + bc.create_table('sent_df', ns_df) sentences = sentences.persist() wait(sentences) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index b1d3a528..a41b670a 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -148,6 +148,6 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config, create_blazing_context=True) + client, _ = attach_to_cluster(config) c = Context() run_query(config=config, client=client, query_func=main, blazing_context=c) From 2de0db505666de20b66a1c27edab428191eec904 Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Mon, 16 Aug 2021 13:58:13 -0700 Subject: [PATCH 03/51] Specify web_clickstreams columns --- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 10 +++++++++- gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 9 ++++++++- gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py | 3 ++- gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 5 +++++ 4 files changed, 24 insertions(+), 3 deletions(-) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 7b89eac1..34c41b75 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -137,8 +137,16 @@ def read_tables(data_dir, bc): ) item_cols = ["i_category_id", "i_item_sk"] + wcs_cols = [ + "wcs_user_sk", + "wcs_click_time_sk", + "wcs_click_date_sk", + "wcs_item_sk", + "wcs_sales_sk", + ] + item_df = table_reader.read("item", relevant_cols=item_cols) - wcs_df = table_reader.read("web_clickstreams") + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) bc.create_table("web_clickstreams", wcs_df) bc.create_table("item", item_df) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 878d5767..dc89cd33 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -155,11 +155,18 @@ def read_tables(data_dir, bc): date_dim_cols = ["d_date_sk", "d_date"] web_page_cols = ["wp_web_page_sk", "wp_type"] web_sales_cols = ["ws_net_paid", "ws_order_number", "ws_sold_date_sk"] + wcs_cols = [ + "wcs_user_sk", + "wcs_sales_sk", + "wcs_click_date_sk", + "wcs_click_time_sk", + "wcs_web_page_sk", + ] date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) web_page_df = table_reader.read("web_page", relevant_cols=web_page_cols) web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) - wcs_df = table_reader.read("web_clickstreams") + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) bc.create_table("web_clickstreams", wcs_df) bc.create_table("web_sales", web_sales_df) diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index 85bc01ec..ce1e59ce 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -35,6 +35,7 @@ item_cols = ["i_item_sk", "i_category"] store_sales_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_customer_sk"] +wcs_cols = ["wcs_user_sk", "wcs_click_date_sk", "wcs_item_sk", "wcs_sales_sk"] def read_tables(data_dir, bc): @@ -46,7 +47,7 @@ def read_tables(data_dir, bc): item_df = table_reader.read("item", relevant_cols=item_cols) store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - wcs_df = table_reader.read("web_clickstreams") + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) bc.create_table("web_clickstreams", wcs_df) bc.create_table("store_sales", store_sales_df) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 2a0464ac..9ae8b352 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -59,6 +59,7 @@ def read_tables(data_dir, bc): bc.create_table('web_clickstreams', wcs_df) bc.create_table('item', item_df) + # print(len(wcs_df)) # bc.create_table('web_clickstreams', os.path.join(data_dir, "web_clickstreams/*.parquet")) # bc.create_table('item', os.path.join(data_dir, "item/*.parquet")) @@ -77,6 +78,8 @@ def main(data_dir, client, bc, config): item_df = item_df.persist() wait(item_df) bc.create_table("item_df", item_df) + # print(len(item_df)) + # print(len(item_df.columns)) query_2 = """ SELECT wcs_user_sk, @@ -89,6 +92,7 @@ def main(data_dir, client, bc, config): ORDER BY wcs.wcs_user_sk, tstamp_inSec, i_category_id """ merged_df = bc.sql(query_2) + # print(len(merged_df)) bc.drop_table("item_df") del item_df @@ -108,6 +112,7 @@ def main(data_dir, client, bc, config): pair_df = pair_df.persist() wait(pair_df) bc.create_table('pair_df', pair_df) + # print(len(pair_df)) last_query = f""" SELECT CAST(category_id_1 AS BIGINT) AS category_id_1, From 0f20be6e18f68348247dccc99845edb43143b89c Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Tue, 17 Aug 2021 13:11:12 -0700 Subject: [PATCH 04/51] Set persist=False when creating tables --- .../queries/q01/gpu_bdb_query_01_dask_sql.py | 6 +++--- .../queries/q02/gpu_bdb_query_02_dask_sql.py | 5 +++-- .../queries/q03/gpu_bdb_query_03_dask_sql.py | 8 ++++---- .../queries/q04/gpu_bdb_query_04_dask_sql.py | 6 +++--- .../queries/q05/gpu_bdb_query_05_dask_sql.py | 9 +++++---- .../queries/q06/gpu_bdb_query_06_dask_sql.py | 8 ++++---- .../queries/q07/gpu_bdb_query_07_dask_sql.py | 10 +++++----- .../queries/q08/gpu_bdb_query_08_dask_sql.py | 12 +++++------ .../queries/q09/gpu_bdb_query_09_dask_sql.py | 10 +++++----- .../queries/q10/gpu_bdb_query_10_dask_sql.py | 16 +++++++-------- .../queries/q11/gpu_bdb_query_11_dask_sql.py | 6 +++--- .../queries/q12/gpu_bdb_query_12_dask_sql.py | 6 +++--- .../queries/q13/gpu_bdb_query_13_dask_sql.py | 12 +++++------ .../queries/q14/gpu_bdb_query_14_dask_sql.py | 8 ++++---- .../queries/q15/gpu_bdb_query_15_dask_sql.py | 6 +++--- .../queries/q16/gpu_bdb_query_16_dask_sql.py | 10 +++++----- .../queries/q17/gpu_bdb_query_17_dask_sql.py | 14 ++++++------- .../queries/q18/gpu_bdb_query_18_dask_sql.py | 20 +++++++++---------- .../queries/q19/gpu_bdb_query_19_dask_sql.py | 16 +++++++-------- .../queries/q20/gpu_bdb_query_20_dask_sql.py | 4 ++-- .../queries/q21/gpu_bdb_query_21_dask_sql.py | 12 +++++------ .../queries/q22/gpu_bdb_query_22_dask_sql.py | 8 ++++---- .../queries/q23/gpu_bdb_query_23_dask_sql.py | 12 +++++------ .../queries/q24/gpu_bdb_query_24_dask_sql.py | 8 ++++---- .../queries/q25/gpu_bdb_query_25_dask_sql.py | 6 +++--- .../queries/q26/gpu_bdb_query_26_dask_sql.py | 4 ++-- .../queries/q27/gpu_bdb_query_27_dask_sql.py | 6 +++--- .../queries/q28/gpu_bdb_query_28_dask_sql.py | 2 +- .../queries/q29/gpu_bdb_query_29_dask_sql.py | 6 +++--- .../queries/q30/gpu_bdb_query_30_dask_sql.py | 8 ++++---- 30 files changed, 133 insertions(+), 131 deletions(-) diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index 11ff194d..b6c91b2d 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -54,8 +54,8 @@ def read_tables(data_dir, bc): item_df = table_reader.read("item", relevant_cols=item_cols) ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - bc.create_table("item", item_df) - bc.create_table("store_sales", ss_df) + bc.create_table("item", item_df, persist=False) + bc.create_table("store_sales", ss_df, persist=False) # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) @@ -75,7 +75,7 @@ def main(data_dir, client, bc, config): result_distinct = result_distinct.persist() wait(result_distinct) - bc.create_table("distinct_table", result_distinct) + bc.create_table("distinct_table", result_distinct, persist=False) query = f""" SELECT item_sk_1, item_sk_2, COUNT(*) AS cnt diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index 149b660e..f7f1a7c4 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -47,7 +47,7 @@ def read_tables(data_dir, bc): wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df) + bc.create_table("web_clickstreams", wcs_df, persist=False) # bc.create_table("web_clickstreams", # os.path.join(data_dir, "web_clickstreams/*.parquet")) @@ -76,7 +76,8 @@ def main(data_dir, client, bc, config): session_df = session_df.persist() wait(session_df) - bc.create_table('session_df', session_df) + bc.create_table('session_df', session_df, persist=False) + print(len(session_df)) last_query = f""" WITH item_df AS ( diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 34c41b75..cf7c1061 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -148,8 +148,8 @@ def read_tables(data_dir, bc): item_df = table_reader.read("item", relevant_cols=item_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df) - bc.create_table("item", item_df) + bc.create_table("web_clickstreams", wcs_df, persist=False) + bc.create_table("item", item_df, persist=False) def main(data_dir, client, bc, config): @@ -164,7 +164,7 @@ def main(data_dir, client, bc, config): item_df = item_df.persist() wait(item_df) - bc.create_table("item_df", item_df) + bc.create_table("item_df", item_df, persist=False) query_2 = """ SELECT CAST(w.wcs_user_sk AS INTEGER) as wcs_user_sk, @@ -198,7 +198,7 @@ def main(data_dir, client, bc, config): del merged_df del item_df_filtered - bc.create_table('product_result', product_view_results) + bc.create_table('product_result', product_view_results, persist=False) last_query = f""" SELECT CAST({q03_purchased_item_IN} AS BIGINT) AS purchased_item, diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index 725cd5da..5ae081a2 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -105,8 +105,8 @@ def read_tables(data_dir, bc): ] wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table('web_page_wo_categorical', wp_df) - bc.create_table('web_clickstreams', wcs_df) + bc.create_table('web_page_wo_categorical', wp_df, persist=False) + bc.create_table('web_clickstreams', wcs_df, persist=False) # bc.create_table('web_page_wo_categorical', os.path.join(data_dir, "web_page/*.parquet")) # bc.create_table('web_clickstreams', @@ -144,7 +144,7 @@ def main(data_dir, client, bc, config): wp = wp.persist() wait(wp) - bc.create_table('web_page', wp) + bc.create_table('web_page', wp, persist=False) query = """ SELECT diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index af90eded..caef3c04 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -67,10 +67,10 @@ def read_tables(data_dir, bc): "web_clickstreams", relevant_cols=wcs_columns, index=False ) - bc.create_table("web_clickstreams", wcs_ddf) - bc.create_table("customer", customer_ddf) - bc.create_table("item", item_ddf) - bc.create_table("customer_demographics", customer_dem_ddf) + bc.create_table("web_clickstreams", wcs_ddf, persist=False) + bc.create_table("customer", customer_ddf, persist=False) + bc.create_table("item", item_ddf, persist=False) + bc.create_table("customer_demographics", customer_dem_ddf, persist=False) # bc.create_table("web_clickstreams", os.path.join(data_dir, "web_clickstreams/*.parquet")) # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) @@ -163,6 +163,7 @@ def main(data_dir, client, bc, config): """ cust_and_clicks_ddf = bc.sql(query) + print(len(cust_and_clicks_ddf)) cust_and_clicks_ddf = cust_and_clicks_ddf.repartition(npartitions=1) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index 69b8abe7..6cda07be 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -76,10 +76,10 @@ def read_tables(data_dir, bc): date_df = table_reader.read("date_dim", relevant_cols=date_cols) customer_df = table_reader.read("customer", relevant_cols=customer_cols) - bc.create_table('web_sales', ws_df) - bc.create_table('store_sales', ss_df) - bc.create_table('date_dim', date_df) - bc.create_table('customer', customer_df) + bc.create_table('web_sales', ws_df, persist=False) + bc.create_table('store_sales', ss_df, persist=False) + bc.create_table('date_dim', date_df, persist=False) + bc.create_table('customer', customer_df, persist=False) # bc.create_table('web_sales', os.path.join(data_dir, "web_sales/*.parquet")) # bc.create_table('store_sales', os.path.join(data_dir, "store_sales/*.parquet")) diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index f49f75cb..204c13a9 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -54,11 +54,11 @@ def read_tables(data_dir, bc): "customer_address", relevant_cols=customer_address_cols ) - bc.create_table("item", item_df) - bc.create_table("customer", customer_df) - bc.create_table("store_sales", store_sales_df) - bc.create_table("date_dim", date_dim_df) - bc.create_table("customer_address", customer_address_df) + bc.create_table("item", item_df, persist=False) + bc.create_table("customer", customer_df, persist=False) + bc.create_table("store_sales", store_sales_df, persist=False) + bc.create_table("date_dim", date_dim_df, persist=False) + bc.create_table("customer_address", customer_address_df, persist=False) # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index dc89cd33..8763fe8f 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -168,10 +168,10 @@ def read_tables(data_dir, bc): web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df) - bc.create_table("web_sales", web_sales_df) - bc.create_table("web_page", web_page_df) - bc.create_table("date_dim", date_dim_df) + bc.create_table("web_clickstreams", wcs_df, persist=False) + bc.create_table("web_sales", web_sales_df, persist=False) + bc.create_table("web_page", web_page_df, persist=False) + bc.create_table("date_dim", date_dim_df, persist=False) # bc.create_table("web_clickstreams", os.path.join(data_dir, "web_clickstreams/*.parquet")) # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) @@ -223,7 +223,7 @@ def main(data_dir, client, bc, config): web_page_df = web_page_df.persist() wait(web_page_df) - bc.create_table('web_page_2', web_page_df) + bc.create_table('web_page_2', web_page_df, persist=False) query_2 = f""" SELECT @@ -260,7 +260,7 @@ def main(data_dir, client, bc, config): unique_review_sales = unique_review_sales.persist() wait(unique_review_sales) - bc.create_table("reviews", unique_review_sales) + bc.create_table("reviews", unique_review_sales, persist=False) last_query = f""" SELECT CAST(review_total AS BIGINT) AS q08_review_sales_amount, diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index d13dc302..7fec51f4 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -94,11 +94,11 @@ def read_tables(data_dir, bc): s_columns = ["s_store_sk"] store = table_reader.read("store", relevant_cols=s_columns) - bc.create_table("store_sales", store_sales) - bc.create_table("customer_address", customer_address) - bc.create_table("customer_demographics", customer_demographics) - bc.create_table("date_dim", date_dim) - bc.create_table("store", store) + bc.create_table("store_sales", store_sales, persist=False) + bc.create_table("customer_address", customer_address, persist=False) + bc.create_table("customer_demographics", customer_demographics, persist=False) + bc.create_table("date_dim", date_dim, persist=False) + bc.create_table("store", store, persist=False) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) # bc.create_table("customer_address", os.path.join(data_dir, "customer_address/*.parquet")) diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index da1cb162..58650d55 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -56,7 +56,7 @@ def read_tables(data_dir, bc): "product_reviews", relevant_cols=product_reviews_cols, ) - bc.create_table("product_reviews", product_reviews_df) + bc.create_table("product_reviews", product_reviews_df, persist=False) # bc.create_table('product_reviews', os.path.join(data_dir, "product_reviews/*.parquet")) @@ -100,24 +100,24 @@ def main(data_dir, client, bc, config): product_reviews_df = product_reviews_df.persist() wait(product_reviews_df) - bc.create_table('product_reviews_df', product_reviews_df) + bc.create_table('product_reviews_df', product_reviews_df, persist=False) sentences = sentences.persist() wait(sentences) - bc.create_table('sentences', sentences) + bc.create_table('sentences', sentences, persist=False) # These files come from the official TPCx-BB kit # We extracted them from bigbenchqueriesmr.jar # Need to pass the absolute path for these txt files sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") - ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) - bc.create_table('negative_sentiment', ns_df) - ps_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "positiveSentiment.txt"), names=["sentiment_word"]) - bc.create_table('positive_sentiment', ps_df) + ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"], persist=False) + bc.create_table('negative_sentiment', ns_df, persist=False) + ps_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "positiveSentiment.txt"), names=["sentiment_word"], persist=False) + bc.create_table('positive_sentiment', ps_df, persist=False) word_df = word_df.persist() wait(word_df) - bc.create_table('word_df', word_df) + bc.create_table('word_df', word_df, persist=False) query = ''' SELECT pr_item_sk as item_sk, diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index bece80ff..730c9bbe 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -56,9 +56,9 @@ def read_tables(data_dir, bc): ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) date_df = table_reader.read("date_dim", relevant_cols=date_cols) - bc.create_table("web_sales", ws_df) - bc.create_table("product_reviews", pr_df) - bc.create_table("date_dim", date_df) + bc.create_table("web_sales", ws_df, persist=False) + bc.create_table("product_reviews", pr_df, persist=False) + bc.create_table("date_dim", date_df, persist=False) # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index ce1e59ce..fc292264 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -49,9 +49,9 @@ def read_tables(data_dir, bc): store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df) - bc.create_table("store_sales", store_sales_df) - bc.create_table("item", item_df) + bc.create_table("web_clickstreams", wcs_df, persist=False) + bc.create_table("store_sales", store_sales_df, persist=False) + bc.create_table("item", item_df, persist=False) # bc.create_table("web_clickstreams", # os.path.join(data_dir, "web_clickstreams/*.parquet")) diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index 71109386..4c3487e4 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -52,10 +52,10 @@ def read_tables(data_dir, bc): w_sales_cols = ["ws_sold_date_sk", "ws_bill_customer_sk", "ws_net_paid"] web_sales_df = table_reader.read("web_sales", relevant_cols=w_sales_cols) - bc.create_table("date_dim", date_dim_df) - bc.create_table("customer", customer_df) - bc.create_table("store_sales", s_sales_df) - bc.create_table("web_sales", web_sales_df) + bc.create_table("date_dim", date_dim_df, persist=False) + bc.create_table("customer", customer_df, persist=False) + bc.create_table("store_sales", s_sales_df, persist=False) + bc.create_table("web_sales", web_sales_df, persist=False) # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) @@ -85,7 +85,7 @@ def main(data_dir, client, bc, config): temp_table1 = temp_table1.persist() wait(temp_table1) - bc.create_table("temp_table1", temp_table1) + bc.create_table("temp_table1", temp_table1, persist=False) query_2 = """ SELECT ws.ws_bill_customer_sk AS customer_sk, @@ -105,7 +105,7 @@ def main(data_dir, client, bc, config): temp_table2 = temp_table2.persist() wait(temp_table2) - bc.create_table("temp_table2", temp_table2) + bc.create_table("temp_table2", temp_table2, persist=False) query = """ SELECT CAST(c_customer_sk AS BIGINT) as c_customer_sk, diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index 3a1e5e25..0e835605 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -51,10 +51,10 @@ def read_tables(data_dir, bc): td_columns = ["t_time_sk", "t_hour"] time_dim = table_reader.read("time_dim", relevant_cols=td_columns) - bc.create_table("household_demographics", household_demographics) - bc.create_table("web_page", web_page) - bc.create_table("web_sales", web_sales) - bc.create_table("time_dim", time_dim) + bc.create_table("household_demographics", household_demographics, persist=False) + bc.create_table("web_page", web_page, persist=False) + bc.create_table("web_sales", web_sales, persist=False) + bc.create_table("time_dim", time_dim, persist=False) # bc.create_table( # "household_demographics", os.path.join(data_dir, "household_demographics/*.parquet" diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index b3be1c41..7f67f8bb 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -53,9 +53,9 @@ def read_tables(data_dir, bc): date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) item_df = table_reader.read("item", relevant_cols=item_cols) - bc.create_table("store_sales", store_sales_df) - bc.create_table("date_dim", date_dim_df) - bc.create_table("item", item_df) + bc.create_table("store_sales", store_sales_df, persist=False) + bc.create_table("date_dim", date_dim_df, persist=False) + bc.create_table("item", item_df, persist=False) def main(data_dir, client, bc, config): diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 36f4059d..11db71ab 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -57,11 +57,11 @@ def read_tables(data_dir, bc): item_df = table_reader.read("item", relevant_cols=item_cols) warehouse_df = table_reader.read("warehouse", relevant_cols=warehouse_cols) - bc.create_table("web_sales", web_sales_df) - bc.create_table("web_returns", web_returns_df) - bc.create_table("date_dim", date_dim_df) - bc.create_table("item", item_df) - bc.create_table("warehouse", warehouse_df) + bc.create_table("web_sales", web_sales_df, persist=False) + bc.create_table("web_returns", web_returns_df, persist=False) + bc.create_table("date_dim", date_dim_df, persist=False) + bc.create_table("item", item_df, persist=False) + bc.create_table("warehouse", warehouse_df, persist=False) # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) # bc.create_table("web_returns", os.path.join(data_dir, "web_returns/*.parquet")) diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 7dca433d..7d7c3eee 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -69,13 +69,13 @@ def read_tables(data_dir, bc): ) promotion_df = table_reader.read("promotion", relevant_cols=promotion_cols) - bc.create_table("store_sales", store_sales_df) - bc.create_table("item", item_df) - bc.create_table("customer", customer_df) - bc.create_table("store", store_df) - bc.create_table("date_dim", date_dim_df) - bc.create_table("customer_address", customer_address_df) - bc.create_table("promotion", promotion_df) + bc.create_table("store_sales", store_sales_df, persist=False) + bc.create_table("item", item_df, persist=False) + bc.create_table("customer", customer_df, persist=False) + bc.create_table("store", store_df, persist=False) + bc.create_table("date_dim", date_dim_df, persist=False) + bc.create_table("customer_address", customer_address_df, persist=False) + bc.create_table("promotion", promotion_df, persist=False) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index e6700d44..efd5561a 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -150,10 +150,10 @@ def read_tables(data_dir, bc): "product_reviews", relevant_cols=product_reviews_cols, ) - bc.create_table("store", store) - bc.create_table("store_sales", store_sales) - bc.create_table("date_dim", date_dim) - bc.create_table("product_reviews", product_reviews) + bc.create_table("store", store, persist=False) + bc.create_table("store_sales", store_sales, persist=False) + bc.create_table("date_dim", date_dim, persist=False) + bc.create_table("product_reviews", product_reviews, persist=False) # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) @@ -255,11 +255,11 @@ def main(data_dir, client, bc, config): stores_with_regression = stores_with_regression.persist() wait(stores_with_regression) - bc.create_table("stores_with_regression", stores_with_regression) + bc.create_table("stores_with_regression", stores_with_regression, persist=False) combined = combined.persist() wait(combined) - bc.create_table("combined", combined) + bc.create_table("combined", combined, persist=False) query_3 = """ SELECT store_ID, @@ -294,19 +294,19 @@ def main(data_dir, client, bc, config): # Need to pass the absolute path for this txt file sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) - bc.create_table('sent_df', ns_df) + bc.create_table('sent_df', ns_df, persist=False) word_df = word_df.persist() wait(word_df) - bc.create_table("word_df", word_df) + bc.create_table("word_df", word_df, persist=False) sentences = sentences.persist() wait(sentences) - bc.create_table("sentences", sentences) + bc.create_table("sentences", sentences, persist=False) temp_table2 = temp_table2.persist() wait(temp_table2) - bc.create_table("temp_table2", temp_table2) + bc.create_table("temp_table2", temp_table2, persist=False) query_4 = """ WITH sentences_table AS diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index 661552e5..7d19dd3d 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -70,10 +70,10 @@ def read_tables(data_dir, bc): "product_reviews", relevant_cols=product_reviews_cols ) - bc.create_table('web_returns', web_returns_df) - bc.create_table('date_dim', date_dim_df) - bc.create_table('product_reviews', product_reviews_df) - bc.create_table('store_returns', store_returns_df) + bc.create_table('web_returns', web_returns_df, persist=False) + bc.create_table('date_dim', date_dim_df, persist=False) + bc.create_table('product_reviews', product_reviews_df, persist=False) + bc.create_table('store_returns', store_returns_df, persist=False) # bc.create_table('web_returns', os.path.join(data_dir, "web_returns/*.parquet")) # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) @@ -153,19 +153,19 @@ def main(data_dir, client, bc, config): # Need to pass the absolute path for this txt file sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) - bc.create_table('sent_df', ns_df) + bc.create_table('sent_df', ns_df, persist=False) sentences = sentences.persist() wait(sentences) - bc.create_table('sentences_df', sentences) + bc.create_table('sentences_df', sentences, persist=False) word_df = word_df.persist() wait(word_df) - bc.create_table('word_df', word_df) + bc.create_table('word_df', word_df, persist=False) merged_df = merged_df.persist() wait(merged_df) - bc.create_table('merged_df', merged_df) + bc.create_table('merged_df', merged_df, persist=False) query = """ WITH negativesent AS diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index 503dca03..8bc7c3de 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -89,8 +89,8 @@ def read_tables(data_dir, bc): "store_returns", relevant_cols=store_returns_cols ) - bc.create_table("store_sales", store_sales_df) - bc.create_table("store_returns", store_returns_df) + bc.create_table("store_sales", store_sales_df, persist=False) + bc.create_table("store_returns", store_returns_df, persist=False) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) # bc.create_table("store_returns", os.path.join(data_dir, "store_returns/*.parquet")) diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index 0ed1fffe..47647768 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -64,12 +64,12 @@ def read_tables(data_dir, bc): store_table_df = table_reader.read("store", relevant_cols=store_cols) item_table_df = table_reader.read("item", relevant_cols=item_cols) - bc.create_table("store_sales", store_sales_df) - bc.create_table("date_dim", date_dim_df) - bc.create_table("item", item_table_df) - bc.create_table("web_sales", web_sales_df) - bc.create_table("store_returns", store_returns_df) - bc.create_table("store", store_table_df) + bc.create_table("store_sales", store_sales_df, persist=False) + bc.create_table("date_dim", date_dim_df, persist=False) + bc.create_table("item", item_table_df, persist=False) + bc.create_table("web_sales", web_sales_df, persist=False) + bc.create_table("store_returns", store_returns_df, persist=False) + bc.create_table("store", store_table_df, persist=False) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index 6ba0e5b0..a0d21d4b 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -59,10 +59,10 @@ def read_tables(data_dir, bc): dd_columns = ["d_date_sk", "d_date"] date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) - bc.create_table('inventory', inventory) - bc.create_table('item', item) - bc.create_table('warehouse', warehouse) - bc.create_table('date_dim', date_dim) + bc.create_table('inventory', inventory, persist=False) + bc.create_table('item', item, persist=False) + bc.create_table('warehouse', warehouse, persist=False) + bc.create_table('date_dim', date_dim, persist=False) # bc.create_table('inventory', os.path.join(data_dir, "inventory/*.parquet")) # bc.create_table('item', os.path.join(data_dir, "item/*.parquet")) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index 610ca385..757943e4 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -54,8 +54,8 @@ def read_tables(data_dir, bc): ] inv_df = table_reader.read("inventory", relevant_cols=inv_cols) - bc.create_table('inventory', inv_df) - bc.create_table('date_dim', date_df) + bc.create_table('inventory', inv_df, persist=False) + bc.create_table('date_dim', date_df, persist=False) # bc.create_table('inventory', os.path.join(data_dir, "inventory/*.parquet")) # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) @@ -78,7 +78,7 @@ def main(data_dir, client, bc, config): inv_dates_result = inv_dates_result.persist() wait(inv_dates_result) - bc.create_table('inv_dates', inv_dates_result) + bc.create_table('inv_dates', inv_dates_result, persist=False) query_2 = """ SELECT inv_warehouse_sk, inv_item_sk, @@ -91,7 +91,7 @@ def main(data_dir, client, bc, config): mean_result = mean_result.persist() wait(mean_result) - bc.create_table('mean_df', mean_result) + bc.create_table('mean_df', mean_result, persist=False) query_3 = """ SELECT id.inv_warehouse_sk, id.inv_item_sk, @@ -116,7 +116,7 @@ def main(data_dir, client, bc, config): std_result = std_result.persist() wait(std_result) - bc.create_table('iteration', std_result) + bc.create_table('iteration', std_result, persist=False) query_4 = f""" SELECT inv_warehouse_sk, inv_item_sk, @@ -131,7 +131,7 @@ def main(data_dir, client, bc, config): std_result = std_result.persist() wait(std_result) - bc.create_table('temp_table', std_result) + bc.create_table('temp_table', std_result, persist=False) query = f""" SELECT inv1.inv_warehouse_sk, inv1.inv_item_sk, diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index 1c7802e9..9aada23f 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -53,10 +53,10 @@ def read_tables(data_dir, bc): imp_df = table_reader.read("item_marketprices", relevant_cols=imp_cols) ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - bc.create_table("web_sales", ws_df) - bc.create_table("item", item_df) - bc.create_table("item_marketprices", imp_df) - bc.create_table("store_sales", ss_df) + bc.create_table("web_sales", ws_df, persist=False) + bc.create_table("item", item_df, persist=False) + bc.create_table("item_marketprices", imp_df, persist=False) + bc.create_table("store_sales", ss_df, persist=False) # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index a41b670a..a6f5b83f 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -83,9 +83,9 @@ def read_tables(data_dir, bc): ws_ddf = table_reader.read("web_sales", relevant_cols=ws_cols, index=False) datedim_ddf = table_reader.read("date_dim", relevant_cols=datedim_cols, index=False) - bc.create_table("web_sales", ws_ddf) - bc.create_table("store_sales", ss_ddf) - bc.create_table("date_dim", datedim_ddf) + bc.create_table("web_sales", ws_ddf, persist=False) + bc.create_table("store_sales", ss_ddf, persist=False) + bc.create_table("date_dim", datedim_ddf, persist=False) # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index ec0c4203..1cb614b4 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -77,8 +77,8 @@ def read_tables(data_dir, bc): ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) - bc.create_table("store_sales", ss_ddf) - bc.create_table("item", items_ddf) + bc.create_table("store_sales", ss_ddf, persist=False) + bc.create_table("item", items_ddf, persist=False) # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index 9d4ec599..487ee1f6 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -56,7 +56,7 @@ def read_tables(data_dir, bc): "product_reviews", relevant_cols=product_reviews_cols ) - bc.create_table("product_reviews", product_reviews_df) + bc.create_table("product_reviews", product_reviews_df, persist=False) # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) @@ -120,11 +120,11 @@ def main(data_dir, client, bc, config): # recombine repeated_names = repeated_names.persist() wait(repeated_names) - bc.create_table('repeated_names', repeated_names) + bc.create_table('repeated_names', repeated_names, persist=False) ner_parsed = ner_parsed.persist() wait(ner_parsed) - bc.create_table('ner_parsed', ner_parsed) + bc.create_table('ner_parsed', ner_parsed, persist=False) query = f""" SELECT review_idx_global_pos as review_sk, diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 16b8a28d..0fba924a 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -316,7 +316,7 @@ def read_tables(data_dir, bc): ] pr_df = table_reader.read("product_reviews", relevant_cols=columns) - bc.create_table("product_reviews", pr_df) + bc.create_table("product_reviews", pr_df, persist=False) # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 8015916a..5f8547dc 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -46,8 +46,8 @@ def read_tables(data_dir, bc): ws_cols = ["ws_order_number", "ws_item_sk"] ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) - bc.create_table('item', item_df) - bc.create_table('web_sales', ws_df) + bc.create_table('item', item_df, persist=False) + bc.create_table('web_sales', ws_df, persist=False) def main(data_dir, client, bc, config): @@ -63,7 +63,7 @@ def main(data_dir, client, bc, config): result_distinct = result_distinct.persist() wait(result_distinct) - bc.create_table('distinct_table', result_distinct) + bc.create_table('distinct_table', result_distinct, persist=False) query = f""" SELECT category_id_1, category_id_2, COUNT (*) AS cnt diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 9ae8b352..adbd3aed 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -57,8 +57,8 @@ def read_tables(data_dir, bc): wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table('web_clickstreams', wcs_df) - bc.create_table('item', item_df) + bc.create_table('web_clickstreams', wcs_df, persist=False) + bc.create_table('item', item_df, persist=False) # print(len(wcs_df)) # bc.create_table('web_clickstreams', os.path.join(data_dir, "web_clickstreams/*.parquet")) @@ -77,7 +77,7 @@ def main(data_dir, client, bc, config): item_df = item_df.persist() wait(item_df) - bc.create_table("item_df", item_df) + bc.create_table("item_df", item_df, persist=False) # print(len(item_df)) # print(len(item_df.columns)) @@ -111,7 +111,7 @@ def main(data_dir, client, bc, config): pair_df = pair_df.persist() wait(pair_df) - bc.create_table('pair_df', pair_df) + bc.create_table('pair_df', pair_df, persist=False) # print(len(pair_df)) last_query = f""" From c216a3f66cf1e80b800d1021c3212c408111faf1 Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Tue, 17 Aug 2021 14:13:07 -0700 Subject: [PATCH 05/51] Remove debugging code --- gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 1 - 1 file changed, 1 deletion(-) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index f7f1a7c4..d7abf234 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -77,7 +77,6 @@ def main(data_dir, client, bc, config): session_df = session_df.persist() wait(session_df) bc.create_table('session_df', session_df, persist=False) - print(len(session_df)) last_query = f""" WITH item_df AS ( From 7b96544111a641d6f2a671270ea81ed6ce53e36f Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Tue, 24 Aug 2021 09:04:57 -0700 Subject: [PATCH 06/51] Cleanup query 5 --- gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py | 1 - 1 file changed, 1 deletion(-) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index caef3c04..c56cb2b1 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -163,7 +163,6 @@ def main(data_dir, client, bc, config): """ cust_and_clicks_ddf = bc.sql(query) - print(len(cust_and_clicks_ddf)) cust_and_clicks_ddf = cust_and_clicks_ddf.repartition(npartitions=1) From 5c1eeb86892823f754a9520f81effebf8c6b8d84 Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Tue, 7 Sep 2021 10:35:28 -0700 Subject: [PATCH 07/51] Add DISTRIBUTE BY Operator --- gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 2 +- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 3 ++- gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 1 + gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 2 +- gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 1 + 5 files changed, 6 insertions(+), 3 deletions(-) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index d7abf234..ec27c7b6 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -63,7 +63,7 @@ def main(data_dir, client, bc, config): FROM web_clickstreams WHERE wcs_item_sk IS NOT NULL AND wcs_user_sk IS NOT NULL - ORDER BY wcs_user_sk + DISTRIBUTE BY wcs_user_sk """ wcs_result = bc.sql(query_1) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index cf7c1061..d6368927 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -175,7 +175,7 @@ def main(data_dir, client, bc, config): INNER JOIN item_df AS i ON w.wcs_item_sk = i.i_item_sk WHERE w.wcs_user_sk IS NOT NULL AND w.wcs_item_sk IS NOT NULL - ORDER BY w.wcs_user_sk + DISTRIBUTE BY wcs_user_sk """ merged_df = bc.sql(query_2) @@ -208,6 +208,7 @@ def main(data_dir, client, bc, config): GROUP BY i_item_sk ORDER BY purchased_item, cnt desc, lastviewed_item LIMIT {q03_limit} + DISTRIBUTE BY lastviewed_item """ result = bc.sql(last_query) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index 5ae081a2..a6d6cb30 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -157,6 +157,7 @@ def main(data_dir, client, bc, config): AND c.wcs_user_sk IS NOT NULL AND c.wcs_sales_sk IS NULL --abandoned implies: no sale ORDER BY wcs_user_sk, tstamp_inSec + DISTRIBUTE BY wcs_user_sk """ merged_df = bc.sql(query) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 8763fe8f..2129848b 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -236,7 +236,7 @@ def main(data_dir, client, bc, config): WHERE wcs_user_sk IS NOT NULL AND wcs_click_date_sk BETWEEN {q08_start_dt} AND {q08_end_dt} --in the future we want to remove this ORDER BY - ORDER BY wcs_user_sk + DISTRIBUTE BY wcs_user_sk """ merged_df = bc.sql(query_2) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index adbd3aed..20505764 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -90,6 +90,7 @@ def main(data_dir, client, bc, config): AND i.i_category_id IS NOT NULL AND wcs.wcs_user_sk IS NOT NULL ORDER BY wcs.wcs_user_sk, tstamp_inSec, i_category_id + DISTRIBUTE BY wcs_user_sk """ merged_df = bc.sql(query_2) # print(len(merged_df)) From 0ba6115c8d5426448f662a293a81adf173643ade Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Wed, 8 Sep 2021 13:37:36 -0700 Subject: [PATCH 08/51] Reset index on train/test df's after sorting --- gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 0fba924a..6a04900f 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -336,6 +336,7 @@ def main(data_dir, client, bc, config): ORDER BY pr_review_sk """ test_data = bc.sql(query1) + test_data = test_data.reset_index(drop=True) # 90 % of data query2 = """ @@ -349,6 +350,7 @@ def main(data_dir, client, bc, config): ORDER BY pr_review_sk """ train_data = bc.sql(query2) + train_data = train_data.reset_index(drop=True) final_data, acc, prec, cmat = post_etl_processing( client=client, train_data=train_data, test_data=test_data From 9a2eacedcbc0b20db16c84b895ba58d47ea4ad23 Mon Sep 17 00:00:00 2001 From: Chris Jarrett Date: Wed, 8 Sep 2021 20:44:19 -0700 Subject: [PATCH 09/51] Fix query 18 --- gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py | 1 + 1 file changed, 1 insertion(+) diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index efd5561a..4f8b2ebf 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -214,6 +214,7 @@ def main(data_dir, client, bc, config): ORDER BY pr_review_date, pr_review_content, pr_review_sk """ no_nulls = bc.sql(query_2) + no_nulls = no_nulls.reset_index(drop=True) targets = ( stores_with_regression.s_store_name.str.lower() From 0b5eb2cdaf16c4de79c9c5ae2991b6e074e9f77f Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Fri, 10 Sep 2021 13:32:47 -0700 Subject: [PATCH 10/51] Merge std_dev aggregation to query2, and remove persist's to reduce memory usage --- .../queries/q23/gpu_bdb_query_23_dask_sql.py | 50 ++++--------------- 1 file changed, 10 insertions(+), 40 deletions(-) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index 757943e4..4682033f 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -76,62 +76,31 @@ def main(data_dir, client, bc, config): """ inv_dates_result = bc.sql(query_1) - inv_dates_result = inv_dates_result.persist() - wait(inv_dates_result) bc.create_table('inv_dates', inv_dates_result, persist=False) query_2 = """ SELECT inv_warehouse_sk, inv_item_sk, d_moy, - AVG(CAST(inv_quantity_on_hand AS DOUBLE)) AS q_mean + AVG(CAST(inv_quantity_on_hand AS DOUBLE)) AS q_mean, + stddev_samp(CAST(inv_quantity_on_hand as DOUBLE)) AS q_std FROM inv_dates GROUP BY inv_warehouse_sk, inv_item_sk, d_moy """ - mean_result = bc.sql(query_2) - - mean_result = mean_result.persist() - wait(mean_result) - bc.create_table('mean_df', mean_result, persist=False) - query_3 = """ - SELECT id.inv_warehouse_sk, - id.inv_item_sk, - id.d_moy, - md.q_mean, - SQRT( SUM( (id.inv_quantity_on_hand - md.q_mean) * (id.inv_quantity_on_hand - md.q_mean) ) - / (COUNT(id.inv_quantity_on_hand) - 1.0)) AS q_std - FROM mean_df md - INNER JOIN inv_dates id ON id.inv_warehouse_sk = md.inv_warehouse_sk - AND id.inv_item_sk = md.inv_item_sk - AND id.d_moy = md.d_moy - AND md.q_mean > 0.0 - GROUP BY id.inv_warehouse_sk, id.inv_item_sk, id.d_moy, md.q_mean - """ - std_result = bc.sql(query_3) - - bc.drop_table("inv_dates") - del inv_dates_result + iteration_1 = bc.sql(query_2) - bc.drop_table("mean_df") - del mean_result - - std_result = std_result.persist() - wait(std_result) - bc.create_table('iteration', std_result, persist=False) - query_4 = f""" + bc.create_table('iteration_1', iteration_1, persist=False) + query_3 = f""" SELECT inv_warehouse_sk, inv_item_sk, d_moy, q_std / q_mean AS qty_cov - FROM iteration + FROM iteration_1 WHERE (q_std / q_mean) >= {q23_coefficient} """ - std_result = bc.sql(query_4) - bc.drop_table("iteration") + iteration_2 = bc.sql(query_3) - std_result = std_result.persist() - wait(std_result) - bc.create_table('temp_table', std_result, persist=False) + bc.create_table('temp_table', iteration_2, persist=False) query = f""" SELECT inv1.inv_warehouse_sk, inv1.inv_item_sk, @@ -148,7 +117,8 @@ def main(data_dir, client, bc, config): inv1.inv_item_sk """ result = bc.sql(query) - + result = result.persist() + wait(result) bc.drop_table("temp_table") return result From ddf0c9f573a5514ab9000430b1968ba0bf99fc79 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Wed, 15 Sep 2021 15:33:10 -0700 Subject: [PATCH 11/51] Fix duplicate index for 5, 8, and 26 --- gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py | 2 ++ gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 2 ++ gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py | 1 + 3 files changed, 5 insertions(+) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index c56cb2b1..3e32f3d2 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -56,6 +56,8 @@ def read_tables(data_dir, bc): ) item_ddf = table_reader.read("item", relevant_cols=items_columns, index=False) + item_ddf = item_ddf.reset_index(drop=True) + customer_ddf = table_reader.read( "customer", relevant_cols=customer_columns, index=False ) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 2129848b..79ae6b3b 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -168,6 +168,8 @@ def read_tables(data_dir, bc): web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + wcs_df = wcs_df.reset_index(drop=True) + bc.create_table("web_clickstreams", wcs_df, persist=False) bc.create_table("web_sales", web_sales_df, persist=False) bc.create_table("web_page", web_page_df, persist=False) diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 1cb614b4..71c011d5 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -76,6 +76,7 @@ def read_tables(data_dir, bc): ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) + items_ddf = items_ddf.reset_index(drop=True) bc.create_table("store_sales", ss_ddf, persist=False) bc.create_table("item", items_ddf, persist=False) From 73ee7a9adf5c83eab9488b3661ae40b3f0523e4f Mon Sep 17 00:00:00 2001 From: Randy Gelhausen Date: Mon, 20 Sep 2021 06:48:09 -0700 Subject: [PATCH 12/51] Updating bdb_tools & test dask-sql queries for shared Context --- gpu_bdb/bdb_tools/cluster_startup.py | 15 +++++---------- gpu_bdb/bdb_tools/utils.py | 2 +- gpu_bdb/benchmark_runner.py | 7 +++++-- gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py | 9 +++------ gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 9 ++++----- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 9 ++++----- gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 9 ++++----- gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py | 9 ++++----- gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 9 ++++----- gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py | 7 +++---- 10 files changed, 37 insertions(+), 48 deletions(-) diff --git a/gpu_bdb/bdb_tools/cluster_startup.py b/gpu_bdb/bdb_tools/cluster_startup.py index e90e2737..ec96625b 100755 --- a/gpu_bdb/bdb_tools/cluster_startup.py +++ b/gpu_bdb/bdb_tools/cluster_startup.py @@ -132,16 +132,11 @@ def maybe_create_worker_directories(dask_worker): config["80GB_workers"] = worker_counts.get("80GB", 0) bc = None + create_blazing_context = True if create_blazing_context: - from blazingsql import BlazingContext - bc = BlazingContext( - dask_client=client, - pool=os.environ.get("BLAZING_POOL", False), - network_interface=os.environ.get("INTERFACE", "ib0"), - config_options=get_bsql_config_options(), - allocator=os.environ.get("BLAZING_ALLOCATOR_MODE", "existing"), - initial_pool_size=os.environ.get("BLAZING_INITIAL_POOL_SIZE", None) - ) + print('Creating context..') + from dask_sql import Context + bc = Context() return client, bc @@ -173,7 +168,7 @@ def _get_ucx_config(): Get a subset of ucx config variables relevant for benchmarking """ relevant_configs = ["infiniband", "nvlink"] - ucx_config = dask.config.get("ucx") + ucx_config = dask.config.get("distributed.comm.ucx") # Doing this since when relevant configs are not enabled the value is `None` instead of `False` filtered_ucx_config = { config: ucx_config.get(config) if ucx_config.get(config) else False diff --git a/gpu_bdb/bdb_tools/utils.py b/gpu_bdb/bdb_tools/utils.py index 3e53cbbe..f807586a 100755 --- a/gpu_bdb/bdb_tools/utils.py +++ b/gpu_bdb/bdb_tools/utils.py @@ -904,7 +904,7 @@ def push_payload_to_googlesheet(config): payload = build_benchmark_googlesheet_payload(config) s = gc.open(config["sheet"]) tab = s.worksheet(config["tab"]) - tab.append_row(payload, value_input_option='USER_ENTERED') + tab.append_row(payload, value_input_option='USER_ENTERED', table_range='A2') ################################# diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index 6fdef1df..0373034e 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -5,7 +5,7 @@ import time import uuid -N_REPEATS = 5 +N_REPEATS = 1 def get_qnum_from_filename(name): @@ -22,7 +22,9 @@ def load_query(qnum, fn): dask_qnums = [str(i).zfill(2) for i in range(1, 31)] +dask_qnums = [] bsql_qnums = [str(i).zfill(2) for i in range(1, 31)] +bsql_qnums = ['01', '05', '08', '26', '27'] if __name__ == "__main__": @@ -39,9 +41,10 @@ def load_query(qnum, fn): for qnum in dask_qnums } + include_blazing = True if include_blazing: bsql_queries = { - qnum: load_query(qnum, f"queries/q{qnum}/gpu_bdb_query_{qnum}_sql.py") + qnum: load_query(qnum, f"queries/q{qnum}/gpu_bdb_query_{qnum}_dask_sql.py") for qnum in bsql_qnums } diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index b6c91b2d..a51447da 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -29,8 +29,6 @@ from dask.distributed import wait -from dask_sql import Context - # -------- Q1 ----------- q01_i_category_id_IN = "1, 2, 3" @@ -44,7 +42,7 @@ ss_cols = ["ss_item_sk", "ss_store_sk", "ss_ticket_number"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -62,7 +60,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_distinct = f""" SELECT DISTINCT ss_item_sk, ss_ticket_number @@ -103,5 +101,4 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() client, bc = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index ec27c7b6..68678ae1 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -38,7 +38,7 @@ q02_session_timeout_inSec = 3600 -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -53,7 +53,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT @@ -107,6 +107,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index d6368927..6b7fd6d4 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -129,7 +129,7 @@ def apply_find_items_viewed(df, item_mappings): return filtered -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -153,7 +153,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT i_item_sk, @@ -219,6 +219,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index a6d6cb30..189b1000 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -86,7 +86,7 @@ def reduction_function(df, keep_cols, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): return df -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -114,7 +114,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_web_page = """ SELECT wp_type, wp_web_page_sk @@ -178,6 +178,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index 3e32f3d2..94c49eac 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -48,7 +48,7 @@ customer_columns = ["c_customer_sk", "c_current_cdemo_sk"] customer_dem_columns = ["cd_demo_sk", "cd_gender", "cd_education_status"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -124,7 +124,7 @@ def build_and_predict_model(ml_input_df): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = """ SELECT @@ -188,6 +188,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 79ae6b3b..a504344d 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -145,7 +145,7 @@ def prep_for_sessionization(df, review_cat_code): return df_filtered -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -182,7 +182,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = f""" SELECT d_date_sk @@ -285,6 +285,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 71c011d5..0772c177 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -64,7 +64,7 @@ def get_clusters(client, kmeans_input_df): return results_dict -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -86,7 +86,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" SELECT @@ -127,6 +127,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() + client, c = attach_to_cluster(config) run_query(config=config, client=client, query_func=main, blazing_context=c) From ed7b137f322c45cbeb6db368c8a36ddc3b7898b6 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Mon, 20 Sep 2021 11:41:09 -0700 Subject: [PATCH 13/51] added split_out to q29 --- .../queries/q29/gpu_bdb_query_29_dask_sql.py | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 5f8547dc..d1f77d22 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -52,17 +52,27 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + n_workers = len(client.scheduler_info()["workers"]) - query_distinct = """ - SELECT DISTINCT i_category_id, ws_order_number + join_query = """ + -- Commented Distinct as we do it in drop_duplicates + -- 553 M rows dont fit on single GPU (int32,int64 column) + -- TODO: Remove when we support Split Out + -- https://github.com/dask-contrib/dask-sql/issues/241 + + SELECT i_category_id, ws_order_number FROM web_sales ws, item i WHERE ws.ws_item_sk = i.i_item_sk AND i.i_category_id IS NOT NULL """ - result_distinct = bc.sql(query_distinct) - - result_distinct = result_distinct.persist() - wait(result_distinct) + result = bc.sql(join_query) + + # Distinct Calculatiin + result_distinct = result.drop_duplicates(split_out=n_workers,ignore_index=True) + ## Remove the int64 index that was created + ## TODO Raise a issue for this + result_distinct = result_distinct.reset_index(drop=True) + bc.create_table('distinct_table', result_distinct, persist=False) query = f""" From a2a52e200da058ee081914b08796ff5e46f56247 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Mon, 20 Sep 2021 14:45:19 -0700 Subject: [PATCH 14/51] Added persist to prevent duplicate computation --- gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index d1f77d22..6543a994 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -55,7 +55,8 @@ def main(data_dir, client, bc, config): n_workers = len(client.scheduler_info()["workers"]) join_query = """ - -- Commented Distinct as we do it in drop_duplicates + -- Commented Distinct as we do it in + -- dask_cudf based drop_duplicates with drop_duplicates -- 553 M rows dont fit on single GPU (int32,int64 column) -- TODO: Remove when we support Split Out -- https://github.com/dask-contrib/dask-sql/issues/241 @@ -72,8 +73,8 @@ def main(data_dir, client, bc, config): ## Remove the int64 index that was created ## TODO Raise a issue for this result_distinct = result_distinct.reset_index(drop=True) - - bc.create_table('distinct_table', result_distinct, persist=False) + ### Persiting cause Order by causes execution + bc.create_table('distinct_table', result_distinct, persist=True) query = f""" SELECT category_id_1, category_id_2, COUNT (*) AS cnt @@ -91,6 +92,8 @@ def main(data_dir, client, bc, config): LIMIT {q29_limit} """ result = bc.sql(query) + result = result.persist() + wait(result); bc.drop_table("distinct_table") return result From c5b40f0e3a10604a97ad3e351f730fba7e03b1de Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Mon, 20 Sep 2021 14:47:12 -0700 Subject: [PATCH 15/51] fixed comment --- gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 6543a994..2c95d2e2 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -56,7 +56,7 @@ def main(data_dir, client, bc, config): join_query = """ -- Commented Distinct as we do it in - -- dask_cudf based drop_duplicates with drop_duplicates + -- dask_cudf based drop_duplicates with split_out -- 553 M rows dont fit on single GPU (int32,int64 column) -- TODO: Remove when we support Split Out -- https://github.com/dask-contrib/dask-sql/issues/241 From 0dc6440b4826bde2be3c9047820dee96dcdb69d8 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Mon, 20 Sep 2021 14:48:24 -0700 Subject: [PATCH 16/51] fixed comment --- gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 2c95d2e2..b1720cca 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -55,7 +55,7 @@ def main(data_dir, client, bc, config): n_workers = len(client.scheduler_info()["workers"]) join_query = """ - -- Commented Distinct as we do it in + -- Removed distinct as we do it in -- dask_cudf based drop_duplicates with split_out -- 553 M rows dont fit on single GPU (int32,int64 column) -- TODO: Remove when we support Split Out From c44374e9c06f855c71ffae1758b18c8f0158d62c Mon Sep 17 00:00:00 2001 From: sft-managed Date: Tue, 21 Sep 2021 09:11:02 -0700 Subject: [PATCH 17/51] Update remaining queries to use shared context --- gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py | 7 +++---- gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py | 7 +++---- gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py | 9 ++++----- gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py | 9 ++++----- gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py | 9 ++++----- gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py | 9 ++++----- gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py | 9 ++++----- gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py | 9 ++++----- gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py | 7 +++---- gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py | 9 ++++----- gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py | 9 ++++----- gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py | 9 ++++----- gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py | 9 ++++----- gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py | 9 ++++----- gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py | 9 ++++----- gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py | 9 ++++----- gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py | 9 ++++----- gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py | 9 ++++----- gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py | 9 ++++----- gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py | 9 ++++----- gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py | 9 ++++----- gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 9 ++++----- gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 9 ++++----- 23 files changed, 89 insertions(+), 112 deletions(-) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index 6cda07be..24284d18 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -36,7 +36,7 @@ q06_YEAR = 2001 -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -88,7 +88,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" WITH temp_table_1 as @@ -158,6 +158,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() client, bc = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index 204c13a9..2f7123f7 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -31,7 +31,7 @@ from dask_sql import Context -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -68,7 +68,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = """ WITH temp_table as @@ -114,5 +114,4 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() client, bc = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index 7fec51f4..5080fd3b 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -61,7 +61,7 @@ q09_part3_sales_price_max = 200 -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -110,7 +110,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" SELECT SUM(ss1.ss_quantity) @@ -178,6 +178,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index 58650d55..62304547 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -42,7 +42,7 @@ eol_char = "รจ" -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): ### splitting by row groups for better parallelism table_reader = build_reader( @@ -62,7 +62,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT pr_item_sk, @@ -165,6 +165,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index 730c9bbe..b3a5e2e3 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -31,7 +31,7 @@ from dask_sql import Context -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -66,7 +66,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = """ WITH p AS @@ -103,6 +103,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index fc292264..7f40ea91 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -38,7 +38,7 @@ wcs_cols = ["wcs_user_sk", "wcs_click_date_sk", "wcs_item_sk", "wcs_sales_sk"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -61,7 +61,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" SELECT DISTINCT wcs_user_sk @@ -97,6 +97,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index 4c3487e4..3660d20d 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -33,7 +33,7 @@ from dask_sql import Context -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -64,7 +64,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT @@ -134,6 +134,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index 0e835605..4efb134d 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -30,7 +30,7 @@ from dask_sql import Context -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -65,7 +65,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = """ SELECT CASE WHEN pmc > 0.0 THEN CAST (amc AS DOUBLE) / CAST (pmc AS DOUBLE) ELSE -1.0 END AS am_pm_ratio @@ -92,6 +92,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index 7f67f8bb..ebbf8721 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -42,7 +42,7 @@ item_cols = ["i_item_sk", "i_category_id"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -59,7 +59,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" SELECT * @@ -98,5 +98,4 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() client, bc = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 11db71ab..204812fc 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -44,7 +44,7 @@ item_cols = ["i_item_sk", "i_item_id"] warehouse_cols = ["w_warehouse_sk", "w_state"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -71,7 +71,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) date = datetime.datetime(2001, 3, 16) start = (date + timedelta(days=-30)).strftime("%Y-%m-%d") @@ -133,6 +133,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 7d7c3eee..5a217bb0 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -52,7 +52,7 @@ customer_address_cols = ["ca_address_sk", "ca_gmt_offset"] promotion_cols = ["p_channel_email", "p_channel_dmail", "p_channel_tv", "p_promo_sk"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -87,7 +87,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_date = f""" select min(d_date_sk) as min_d_date_sk, @@ -136,6 +136,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index 4f8b2ebf..cb14633e 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -121,7 +121,7 @@ def find_relevant_reviews(df, targets, str_col_name="pr_review_content"): return combined -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -162,7 +162,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = f""" WITH temp_table1 AS @@ -358,6 +358,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index 7d19dd3d..d7a19b8b 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -45,7 +45,7 @@ eol_char = "รจ" -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -82,7 +82,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" WITH dateFilter AS @@ -209,6 +209,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index 8bc7c3de..d7d79bc6 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -64,7 +64,7 @@ def get_clusters(client, ml_input_df, feature_cols): return results_dict -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -97,7 +97,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = """ SELECT @@ -163,6 +163,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index 47647768..cf17bd99 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -50,7 +50,7 @@ store_cols = ["s_store_name", "s_store_id", "s_store_sk"] item_cols = ["i_item_id", "i_item_desc", "i_item_sk"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -80,7 +80,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = """ SELECT @@ -168,6 +168,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index a0d21d4b..3079c470 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -36,7 +36,7 @@ q22_i_current_price_max = "1.5" -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -71,7 +71,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" SELECT @@ -118,6 +118,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index 4682033f..feb2d46f 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -38,7 +38,7 @@ q23_coefficient = 1.3 -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -62,7 +62,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = f""" SELECT inv_warehouse_sk, @@ -125,6 +125,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index 9aada23f..f517b78f 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -41,7 +41,7 @@ ] ss_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_quantity"] -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -65,7 +65,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = """ WITH temp_table as @@ -118,6 +118,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index a6f5b83f..e6c41de7 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -63,7 +63,7 @@ def get_clusters(client, ml_input_df): return results_dict -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -93,7 +93,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query = f""" WITH concat_table AS @@ -148,6 +148,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index 487ee1f6..c92fdc0b 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -44,7 +44,7 @@ EOL_CHAR = "." -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): ### splitting by row groups for better parallelism table_reader = build_reader( data_format=config["file_format"], @@ -76,7 +76,7 @@ def ner_parser(df, col_string, batch_size=256): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) import dask_cudf @@ -146,7 +146,6 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 6a04900f..159f0915 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -301,7 +301,7 @@ def post_etl_processing(client, train_data, test_data): return final_data, acc, prec, cmat -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): ### splitting by row groups for better parallelism table_reader = build_reader( data_format=config["file_format"], @@ -322,7 +322,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) # 10 % of data query1 = """ @@ -369,6 +369,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index b1720cca..91b20b31 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -36,7 +36,7 @@ q29_limit = 100 -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -51,7 +51,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) n_workers = len(client.scheduler_info()["workers"]) join_query = """ @@ -101,6 +101,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 20505764..15f2cab0 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -44,7 +44,7 @@ q30_limit = 40 -def read_tables(data_dir, bc): +def read_tables(data_dir, bc, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -66,7 +66,7 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT i_item_sk, @@ -132,6 +132,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, _ = attach_to_cluster(config) - c = Context() - run_query(config=config, client=client, query_func=main, blazing_context=c) + client, bc = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, blazing_context=bc) From 382c7c168e16f95527d9f536010edb815592fa69 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Thu, 23 Sep 2021 08:52:15 -0700 Subject: [PATCH 18/51] Remove extra dask-sql imports --- gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 1 - gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 1 - gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 1 - gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py | 4 ---- gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py | 1 - gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py | 1 - gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 3 --- gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py | 1 - gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py | 1 - gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py | 1 - gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py | 1 - gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py | 1 - gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py | 1 - gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py | 1 - gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py | 1 - gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py | 1 - gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py | 2 -- gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py | 1 - gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py | 1 - gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py | 1 - gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py | 1 - gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py | 1 - gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py | 2 -- gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py | 2 -- gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py | 1 - gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py | 4 ---- gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 1 - gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 1 - 28 files changed, 39 deletions(-) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index 68678ae1..55845620 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -30,7 +30,6 @@ from bdb_tools.sessionization import get_distinct_sessions from dask.distributed import wait -from dask_sql import Context # -------- Q2 ----------- q02_item_sk = 10001 diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 6b7fd6d4..b9205d5d 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -31,7 +31,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q03 ----------- q03_days_in_sec_before_purchase = 864000 diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index 189b1000..fcb9ce33 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -31,7 +31,6 @@ from dask.distributed import wait -from dask_sql import Context def abandonedShoppingCarts(df, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): import cudf diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index 94c49eac..e3aca1d1 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -33,7 +33,6 @@ from sklearn.metrics import roc_auc_score import cupy as cp -from dask_sql import Context # Logistic Regression params # solver = "LBFGS" Used by passing `penalty=None` or "l2" @@ -56,15 +55,12 @@ def read_tables(data_dir, bc, config): ) item_ddf = table_reader.read("item", relevant_cols=items_columns, index=False) - item_ddf = item_ddf.reset_index(drop=True) - customer_ddf = table_reader.read( "customer", relevant_cols=customer_columns, index=False ) customer_dem_ddf = table_reader.read( "customer_demographics", relevant_cols=customer_dem_columns, index=False ) - wcs_ddf = table_reader.read( "web_clickstreams", relevant_cols=wcs_columns, index=False ) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index 24284d18..2eeb1c5f 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context # -------- Q6 ----------- q06_LIMIT = 100 diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index 2f7123f7..56ef9525 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -29,7 +29,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context def read_tables(data_dir, bc, config): table_reader = build_reader( diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index a504344d..606569eb 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -33,7 +33,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q8 ----------- q08_SECONDS_BEFORE_PURCHASE = 259200 @@ -168,8 +167,6 @@ def read_tables(data_dir, bc, config): web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - wcs_df = wcs_df.reset_index(drop=True) - bc.create_table("web_clickstreams", wcs_df, persist=False) bc.create_table("web_sales", web_sales_df, persist=False) bc.create_table("web_page", web_page_df, persist=False) diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index 5080fd3b..1c962df0 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context # -------- Q9 ----------- q09_year = 2001 diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index 62304547..8ff8a883 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -37,7 +37,6 @@ from dask.distributed import wait -from dask_sql import Context eol_char = "รจ" diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index b3a5e2e3..344e20ce 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -29,7 +29,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context def read_tables(data_dir, bc, config): table_reader = build_reader( diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index 7f40ea91..5d27fd18 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context # -------- Q12 ----------- q12_i_category_IN = "'Books', 'Electronics'" diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index 3660d20d..e592345b 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -31,7 +31,6 @@ from dask.distributed import wait -from dask_sql import Context def read_tables(data_dir, bc, config): table_reader = build_reader( diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index 4efb134d..20907afc 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context def read_tables(data_dir, bc, config): table_reader = build_reader( diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index ebbf8721..58218a20 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context # -------- Q15 ----------- # --store_sales date range diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 204812fc..fd466798 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -30,7 +30,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context websale_cols = [ "ws_order_number", diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 5a217bb0..217961f7 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context # ------- Q17 ------ q17_gmt_offset = -5.0 diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index cb14633e..62780872 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -36,7 +36,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q18 ----------- q18_startDate = "2001-05-02" @@ -214,7 +213,6 @@ def main(data_dir, client, bc, config): ORDER BY pr_review_date, pr_review_content, pr_review_sk """ no_nulls = bc.sql(query_2) - no_nulls = no_nulls.reset_index(drop=True) targets = ( stores_with_regression.s_store_name.str.lower() diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index d7a19b8b..064e334a 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -37,7 +37,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q19 ----------- q19_returns_dates_IN = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index cf17bd99..30aeea4e 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context store_sales_cols = [ "ss_item_sk", diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index 3079c470..90da4dff 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context # -------- Q22 ----------- q22_date = "2001-05-08" diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index feb2d46f..a25c7b3a 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -30,7 +30,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q23 ----------- q23_year = 2001 diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index f517b78f..81b6a675 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -28,7 +28,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context ws_cols = ["ws_item_sk", "ws_sold_date_sk", "ws_quantity"] item_cols = ["i_item_sk", "i_current_price"] diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index e6c41de7..e08539a5 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -30,8 +30,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context - # -------- Q25 ----------- # -- store_sales and web_sales date diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 0772c177..70c8c9b1 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -31,7 +31,6 @@ from dask import delayed -from dask_sql import Context # -------- Q26 ----------- q26_i_category_IN = "Books" @@ -76,7 +75,6 @@ def read_tables(data_dir, bc, config): ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) - items_ddf = items_ddf.reset_index(drop=True) bc.create_table("store_sales", ss_ddf, persist=False) bc.create_table("item", items_ddf, persist=False) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index c92fdc0b..b709cf2d 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -37,7 +37,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q27 ----------- q27_pr_item_sk = 10002 diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 159f0915..681ee2f4 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -35,8 +35,6 @@ from bdb_tools.readers import build_reader -from dask_sql import Context - N_FEATURES = 2 ** 23 # Spark is doing 2^20 ngram_range = (1, 2) @@ -336,7 +334,6 @@ def main(data_dir, client, bc, config): ORDER BY pr_review_sk """ test_data = bc.sql(query1) - test_data = test_data.reset_index(drop=True) # 90 % of data query2 = """ @@ -350,7 +347,6 @@ def main(data_dir, client, bc, config): ORDER BY pr_review_sk """ train_data = bc.sql(query2) - train_data = train_data.reset_index(drop=True) final_data, acc, prec, cmat = post_etl_processing( client=client, train_data=train_data, test_data=test_data diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 91b20b31..a2bf7eb6 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -30,7 +30,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q29 ----------- q29_limit = 100 diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 15f2cab0..161228c1 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -35,7 +35,6 @@ from dask.distributed import wait -from dask_sql import Context # -------- Q30 ----------- # session timeout in secs From b71cb5087dcfc1e219c2a20417e15ab0e1f7cbd8 Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Thu, 23 Sep 2021 09:28:24 -0700 Subject: [PATCH 19/51] Fix q22 errros by casting date column to int --- .../queries/q22/gpu_bdb_query_22_dask_sql.py | 51 ++++++++++--------- 1 file changed, 28 insertions(+), 23 deletions(-) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index a0d21d4b..0f1621c1 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -15,6 +15,7 @@ # limitations under the License. # +import numpy as np import sys import os @@ -24,6 +25,7 @@ benchmark, gpubdb_argparser, run_query, + convert_datestring_to_days, ) from bdb_tools.readers import build_reader @@ -58,6 +60,7 @@ def read_tables(data_dir, bc): dd_columns = ["d_date_sk", "d_date"] date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) + date_dim = date_dim.map_partitions(convert_datestring_to_days) bc.create_table('inventory', inventory, persist=False) bc.create_table('item', item, persist=False) @@ -73,14 +76,18 @@ def read_tables(data_dir, bc): def main(data_dir, client, bc, config): benchmark(read_tables, data_dir, bc, dask_profile=config["dask_profile"]) + # Filter limit in days + min_date = np.datetime64(q22_date, "D").astype(int) - 30 + max_date = np.datetime64(q22_date, "D").astype(int) + 30 + d_date_int = np.datetime64(q22_date, "D").astype(int) + ratio_min = 2.0 / 3.0 + ratio_max = 3.0 / 2.0 query = f""" SELECT w_warehouse_name, i_item_id, - SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) - / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) AS inv_before, - SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) - / 1000000 >= 0 THEN inv_quantity_on_hand ELSE 0 END) AS inv_after + SUM(CASE WHEN d_date - {d_date_int} < 0 THEN inv_quantity_on_hand ELSE 0 END) AS inv_before, + SUM(CASE WHEN d_date - {d_date_int} >= 0 THEN inv_quantity_on_hand ELSE 0 END) AS inv_after FROM inventory inv, item i, @@ -90,29 +97,27 @@ def main(data_dir, client, bc, config): AND i_item_sk = inv_item_sk AND inv_warehouse_sk = w_warehouse_sk AND inv_date_sk = d_date_sk - AND timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 >= -30 - AND timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 <= 30 + AND d_date >= {min_date} + AND d_date <= {max_date} GROUP BY w_warehouse_name, i_item_id - HAVING SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date}', CAST(d_date || ' 00:00:00' AS timestamp)) - / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) > 0 - AND - ( - CAST( - SUM (CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 >= 0 THEN inv_quantity_on_hand ELSE 0 END) AS DOUBLE) - / CAST( SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) - AS DOUBLE) >= 0.666667 - ) - AND - ( - CAST( - SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 >= 0 THEN inv_quantity_on_hand ELSE 0 END) AS DOUBLE) - / CAST ( SUM(CASE WHEN timestampdiff(DAY, timestamp '{q22_date} 00:00:00', CAST(d_date || ' 00:00:00' AS timestamp)) / 1000000 < 0 THEN inv_quantity_on_hand ELSE 0 END) - AS DOUBLE) <= 1.50 - ) + """ + intermediate = bc.sql(query) + bc.create_table("intermediate", intermediate ,persist=False) + + query_2 = f""" + SELECT + w_warehouse_name, + i_item_id, + inv_before, + inv_after + FROM intermediate + WHERE inv_before > 0 + AND CAST(inv_after AS DOUBLE) / CAST(inv_before AS DOUBLE) >= {ratio_min} + AND CAST(inv_after AS DOUBLE) / CAST(inv_before AS DOUBLE) <= {ratio_max} ORDER BY w_warehouse_name, i_item_id LIMIT 100 """ - result = bc.sql(query) + result = bc.sql(query_2) return result From 3a26c91e3cf9f5b0d26f33bd6dd5754b2cba9dc9 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Wed, 29 Sep 2021 10:14:46 -0700 Subject: [PATCH 20/51] added Query-25 dask-sql alternate implimentation --- .../queries/q25/gpu_bdb_query_25_dask_sql.py | 148 +++++++++++++----- 1 file changed, 108 insertions(+), 40 deletions(-) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index e08539a5..b9b3a6d5 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -85,53 +85,121 @@ def read_tables(data_dir, bc, config): bc.create_table("store_sales", ss_ddf, persist=False) bc.create_table("date_dim", datedim_ddf, persist=False) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) +def agg_count_distinct(df, group_key, counted_key): + """Returns a Series that is the result of counting distinct instances of 'counted_key' within each 'group_key'. + The series' index will have one entry per unique 'group_key' value. + Workaround for lack of nunique aggregate function on Dask df. + """ + + ### going via repartition for split_out drop duplicates + unique_df = df[[group_key, counted_key]].map_partitions( + lambda df: df.drop_duplicates() + ) + unique_df = unique_df.shuffle(on=[group_key]) + unique_df = unique_df.map_partitions(lambda df: df.drop_duplicates()) + + unique_df = unique_df.groupby(group_key)[counted_key].count() + return unique_df.reset_index(drop=False) def main(data_dir, client, bc, config): benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) - query = f""" - WITH concat_table AS - ( - ( - SELECT - ss_customer_sk AS cid, - count(distinct ss_ticket_number) AS frequency, - max(ss_sold_date_sk) AS most_recent_date, - CAST( SUM(ss_net_paid) AS DOUBLE) AS amount - FROM store_sales ss - JOIN date_dim d ON ss.ss_sold_date_sk = d.d_date_sk - WHERE CAST(d.d_date AS DATE) > DATE '{q25_date}' - AND ss_customer_sk IS NOT NULL - GROUP BY ss_customer_sk - ) union all - ( - SELECT - ws_bill_customer_sk AS cid, - count(distinct ws_order_number) AS frequency, - max(ws_sold_date_sk) AS most_recent_date, - CAST( SUM(ws_net_paid) AS DOUBLE) AS amount - FROM web_sales ws - JOIN date_dim d ON ws.ws_sold_date_sk = d.d_date_sk - WHERE CAST(d.d_date AS DATE) > DATE '{q25_date}' - AND ws_bill_customer_sk IS NOT NULL - GROUP BY ws_bill_customer_sk - ) - ) + q25_date = "2002-01-02" + ss_join_query= f""" + SELECT + ss_customer_sk, + ss_sold_date_sk, + ss_net_paid, + ss_ticket_number + FROM + store_sales ss + JOIN + date_dim d ON ss.ss_sold_date_sk = d.d_date_sk + WHERE + CAST(d.d_date AS DATE) > DATE '{q25_date}' + AND + ss_customer_sk IS NOT NULL + """ + + + ws_join_query = f""" SELECT - cid AS cid, - CASE WHEN 37621 - max(most_recent_date) < 60 THEN 1.0 - ELSE 0.0 END AS recency, -- 37621 == 2003-01-02 - CAST( SUM(frequency) AS BIGINT) AS frequency, --total frequency - CAST( SUM(amount) AS DOUBLE) AS amount --total amount - FROM concat_table - GROUP BY cid - ORDER BY cid + ws_bill_customer_sk, + ws_order_number, + ws_sold_date_sk, + ws_net_paid + FROM + web_sales ws + JOIN + date_dim d ON ws.ws_sold_date_sk = d.d_date_sk + WHERE + CAST(d.d_date AS DATE) > DATE '{q25_date}' + AND + ws_bill_customer_sk IS NOT NULL """ - cluster_input_ddf = bc.sql(query) + + ss_merged_df = bc.sql(ss_join_query) + ws_merged_df = bc.sql(ws_join_query) + + bc.create_table('ss_merged_table', ss_merged_df, persist=False) + bc.create_table('ws_merged_table', ws_merged_df, persist=False) + + ss_agg_query = """ + SELECT + ss_customer_sk AS cid, + -- count(distinct ss_ticket_number) AS frequency, # distinct count groupby OOMS with dask-s + max(ss_sold_date_sk) AS most_recent_date, + CAST( SUM(ss_net_paid) AS DOUBLE) AS amount + FROM ss_merged_table + GROUP BY ss_customer_sk + """ + ws_agg_query= """ + SELECT + ws_bill_customer_sk AS cid, + -- count(distinct ws_order_number) AS frequency, # distinct count groupby OOMS with dask-sql + max(ws_sold_date_sk) AS most_recent_date, + CAST( SUM(ws_net_paid) AS DOUBLE) AS amount + FROM ws_merged_table + GROUP BY ws_bill_customer_sk + """ + + ss_distinct_count_agg = agg_count_distinct(ss_merged_df,'ss_customer_sk','ss_ticket_number') + ss_distinct_count_agg = ss_distinct_count_agg.rename(columns={'ss_customer_sk':'cid', + 'ss_ticket_number':'frequency'}) + ss_agg_df = bc.sql(ss_agg_query) + ### add distinct count + ss_agg_df = ss_agg_df.merge(ss_distinct_count_agg) + + ws_distinct_count_agg = agg_count_distinct(ws_merged_df,'ws_bill_customer_sk','ws_order_number') + ws_distinct_count_agg = ws_distinct_count_agg.rename(columns={'ws_bill_customer_sk':'cid', + 'ws_order_number':'frequency'}) + ws_agg_df = bc.sql(ws_agg_query) + ### add distinct count + ws_agg_df = ws_agg_df.merge(ws_distinct_count_agg) + + bc.create_table('ss_agg_df', ss_agg_df, persist=False) + bc.create_table('ws_agg_df', ws_agg_df, persist=False) + + + result_query = ''' + WITH concat_table AS + ( + SELECT * FROM ss_agg_df + UNION ALL + SELECT * FROM ws_agg_df + ) + SELECT + cid AS cid, + CASE WHEN 37621 - max(most_recent_date) < 60 THEN 1.0 + ELSE 0.0 END AS recency, -- 37621 == 2003-01-02 + CAST( SUM(frequency) AS BIGINT) AS frequency, --total frequency + CAST( SUM(amount) AS DOUBLE) AS amount --total amount + FROM concat_table + GROUP BY cid + ORDER BY cid + ''' + cluster_input_ddf = bc.sql(result_query) # Prepare df for KMeans clustering cluster_input_ddf["recency"] = cluster_input_ddf["recency"].astype("int64") From 558c5de48e929ad123c72abae2455f55bd4b64bd Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Wed, 29 Sep 2021 14:13:52 -0700 Subject: [PATCH 21/51] fixed comment --- gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index b9b3a6d5..7fe53d8a 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -148,7 +148,7 @@ def main(data_dir, client, bc, config): ss_agg_query = """ SELECT ss_customer_sk AS cid, - -- count(distinct ss_ticket_number) AS frequency, # distinct count groupby OOMS with dask-s + -- count(distinct ss_ticket_number) AS frequency, # distinct count groupby OOMS with dask-sql max(ss_sold_date_sk) AS most_recent_date, CAST( SUM(ss_net_paid) AS DOUBLE) AS amount FROM ss_merged_table From fbaa6489673489fcee77fcc4b54568311ec989dc Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Wed, 6 Oct 2021 10:28:18 -0700 Subject: [PATCH 22/51] removed not useful order bys --- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 1 - gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 1 - gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 1 - 3 files changed, 3 deletions(-) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index b9205d5d..14ff540b 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -207,7 +207,6 @@ def main(data_dir, client, bc, config): GROUP BY i_item_sk ORDER BY purchased_item, cnt desc, lastviewed_item LIMIT {q03_limit} - DISTRIBUTE BY lastviewed_item """ result = bc.sql(last_query) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index fcb9ce33..794ff4c9 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -155,7 +155,6 @@ def main(data_dir, client, bc, config): AND c.wcs_web_page_sk IS NOT NULL AND c.wcs_user_sk IS NOT NULL AND c.wcs_sales_sk IS NULL --abandoned implies: no sale - ORDER BY wcs_user_sk, tstamp_inSec DISTRIBUTE BY wcs_user_sk """ merged_df = bc.sql(query) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 161228c1..910205ff 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -88,7 +88,6 @@ def main(data_dir, client, bc, config): WHERE wcs.wcs_item_sk = i.i_item_sk AND i.i_category_id IS NOT NULL AND wcs.wcs_user_sk IS NOT NULL - ORDER BY wcs.wcs_user_sk, tstamp_inSec, i_category_id DISTRIBUTE BY wcs_user_sk """ merged_df = bc.sql(query_2) From c453417cdcc87aa6d2fe0c21d2f2195bfa10a471 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Wed, 6 Oct 2021 10:31:37 -0700 Subject: [PATCH 23/51] remove persist from query-02 --- gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index 55845620..00ead45f 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -73,8 +73,6 @@ def main(data_dir, client, bc, config): ) del wcs_result - session_df = session_df.persist() - wait(session_df) bc.create_table('session_df', session_df, persist=False) last_query = f""" From a0334a29df3d62b77884456cc053b48094474256 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Fri, 8 Oct 2021 11:43:23 -0700 Subject: [PATCH 24/51] q03 removed persist --- .../queries/q03/gpu_bdb_query_03_dask_sql.py | 2 -- .../queries/q30/gpu_bdb_query_30_dask_sql.py | 24 +++++++++++++++---- 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 14ff540b..05cadeb6 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -189,8 +189,6 @@ def main(data_dir, client, bc, config): apply_find_items_viewed, item_mappings=item_df_filtered ) - product_view_results = product_view_results.persist() - wait(product_view_results) bc.drop_table("item_df") del item_df diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 910205ff..bfedc9c0 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -42,6 +42,23 @@ # query output limit q30_limit = 40 +def start_local_cuda_cluster(): + from dask_cuda import LocalCUDACluster + from distributed import Client + + cluster = LocalCUDACluster(local_directory='/data/vjawa/', + device_memory_limit='18 GB', + rmm_pool_size='29 GB', + jit_unspill=True, + enable_nvlink=True) + client = Client(cluster) + return client + + +def start_dask_sql(): + from dask_sql import Context + return Context() + def read_tables(data_dir, bc, config): table_reader = build_reader( @@ -91,7 +108,6 @@ def main(data_dir, client, bc, config): DISTRIBUTE BY wcs_user_sk """ merged_df = bc.sql(query_2) - # print(len(merged_df)) bc.drop_table("item_df") del item_df @@ -108,10 +124,7 @@ def main(data_dir, client, bc, config): output_col_2="category_id_2") del distinct_session_df - pair_df = pair_df.persist() - wait(pair_df) bc.create_table('pair_df', pair_df, persist=False) - # print(len(pair_df)) last_query = f""" SELECT CAST(category_id_1 AS BIGINT) AS category_id_1, @@ -130,5 +143,6 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) + #client, bc = attach_to_cluster(config) + client, bc = start_local_cuda_cluster(), start_dask_sql() run_query(config=config, client=client, query_func=main, blazing_context=bc) From 5b2007802999ebbe7b8f1a77ce51da4005bf61f3 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Fri, 8 Oct 2021 13:10:35 -0700 Subject: [PATCH 25/51] Revert incorrect cluster change --- .../queries/q30/gpu_bdb_query_30_dask_sql.py | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index bfedc9c0..816f5cac 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -42,22 +42,6 @@ # query output limit q30_limit = 40 -def start_local_cuda_cluster(): - from dask_cuda import LocalCUDACluster - from distributed import Client - - cluster = LocalCUDACluster(local_directory='/data/vjawa/', - device_memory_limit='18 GB', - rmm_pool_size='29 GB', - jit_unspill=True, - enable_nvlink=True) - client = Client(cluster) - return client - - -def start_dask_sql(): - from dask_sql import Context - return Context() def read_tables(data_dir, bc, config): @@ -143,6 +127,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - #client, bc = attach_to_cluster(config) - client, bc = start_local_cuda_cluster(), start_dask_sql() + client, bc = attach_to_cluster(config) run_query(config=config, client=client, query_func=main, blazing_context=bc) From 824d5dd10f02b4aaf17b7796b8fd924262216226 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Wed, 3 Nov 2021 15:50:50 -0700 Subject: [PATCH 26/51] Fix query 3 numba issue --- gpu_bdb/benchmark_runner.py | 2 ++ gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 13 +++++++++++++ 2 files changed, 15 insertions(+) diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index 0373034e..0c91588c 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -4,6 +4,7 @@ import gc import time import uuid +import sys N_REPEATS = 1 @@ -18,6 +19,7 @@ def load_query(qnum, fn): loader = importlib.machinery.SourceFileLoader(qnum, fn) mod = types.ModuleType(loader.name) loader.exec_module(mod) + sys.modules[loader.name] = mod return mod.main diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 05cadeb6..97dc12b7 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -78,6 +78,18 @@ def find_items_viewed_before_purchase_kernel( else: out_col[i * N + k - 1] = 0 +def load_q03(): + import importlib, types + + fn = os.path.join(os.getcwd(), "gpu_bdb_query_03_dask_sql.py") + if not os.path.isfile(fn): + fn = os.path.join(os.getcwd(), "queries/q03/gpu_bdb_query_03_dask_sql.py") + + loader = importlib.machinery.SourceFileLoader("03", fn) + mod = types.ModuleType(loader.name) + loader.exec_module(mod) + sys.modules[loader.name] = mod + return mod.main def apply_find_items_viewed(df, item_mappings): import cudf @@ -104,6 +116,7 @@ def apply_find_items_viewed(df, item_mappings): # we know this can be int32, since it's going to contain item_sks out_arr = cuda.device_array(size * N, dtype=df["wcs_item_sk"].dtype) + load_q03() find_items_viewed_before_purchase_kernel.forall(size)( sample["relevant_idx_pos"], df["wcs_user_sk"], From 0e058a61588c2e9e16e11a4f55ea5b4d5edc4223 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Tue, 9 Nov 2021 11:28:16 -0800 Subject: [PATCH 27/51] Fix module load errors --- gpu_bdb/benchmark_runner.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index 0c91588c..0373034e 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -4,7 +4,6 @@ import gc import time import uuid -import sys N_REPEATS = 1 @@ -19,7 +18,6 @@ def load_query(qnum, fn): loader = importlib.machinery.SourceFileLoader(qnum, fn) mod = types.ModuleType(loader.name) loader.exec_module(mod) - sys.modules[loader.name] = mod return mod.main From 65d2ea476f1eac3308f6d3a9d467aa68a2ba6b2f Mon Sep 17 00:00:00 2001 From: Randy Gelhausen Date: Wed, 8 Dec 2021 08:51:13 -0800 Subject: [PATCH 28/51] Variable renaming: bsql->dask-sql --- gpu_bdb/bdb_tools/cluster_startup.py | 45 ++---------- gpu_bdb/bdb_tools/utils.py | 34 ++++----- gpu_bdb/benchmark_runner.py | 71 ++++++++++--------- .../queries/load_test/gpu_bdb_load_test.py | 4 +- .../queries/q01/gpu_bdb_query_01_dask_sql.py | 27 +++---- .../queries/q02/gpu_bdb_query_02_dask_sql.py | 22 +++--- .../queries/q03/gpu_bdb_query_03_dask_sql.py | 30 ++++---- .../queries/q04/gpu_bdb_query_04_dask_sql.py | 26 +++---- .../queries/q05/gpu_bdb_query_05_dask_sql.py | 27 +++---- .../queries/q06/gpu_bdb_query_06_dask_sql.py | 25 +++---- .../queries/q07/gpu_bdb_query_07_dask_sql.py | 28 +++----- .../queries/q08/gpu_bdb_query_08_dask_sql.py | 39 +++++----- .../queries/q09/gpu_bdb_query_09_dask_sql.py | 30 +++----- .../queries/q10/gpu_bdb_query_10_dask_sql.py | 34 +++++---- .../queries/q11/gpu_bdb_query_11_dask_sql.py | 22 +++--- .../queries/q12/gpu_bdb_query_12_dask_sql.py | 24 +++---- .../queries/q13/gpu_bdb_query_13_dask_sql.py | 37 +++++----- .../queries/q14/gpu_bdb_query_14_dask_sql.py | 27 +++---- .../queries/q15/gpu_bdb_query_15_dask_sql.py | 18 ++--- .../queries/q16/gpu_bdb_query_16_dask_sql.py | 30 ++++---- .../queries/q17/gpu_bdb_query_17_dask_sql.py | 36 ++++------ .../queries/q18/gpu_bdb_query_18_dask_sql.py | 53 +++++++------- .../queries/q19/gpu_bdb_query_19_dask_sql.py | 41 +++++------ .../queries/q20/dask-worker-space/global.lock | 0 .../queries/q20/dask-worker-space/purge.lock | 0 .../queries/q20/gpu_bdb_query_20_dask_sql.py | 19 +++-- .../queries/q21/gpu_bdb_query_21_dask_sql.py | 31 ++++---- .../queries/q22/gpu_bdb_query_22_dask_sql.py | 29 ++++---- .../queries/q23/gpu_bdb_query_23_dask_sql.py | 33 ++++----- .../queries/q24/gpu_bdb_query_24_dask_sql.py | 25 +++---- .../queries/q25/gpu_bdb_query_25_dask_sql.py | 34 ++++----- .../queries/q26/gpu_bdb_query_26_dask_sql.py | 17 ++--- .../queries/q27/gpu_bdb_query_27_dask_sql.py | 26 ++++--- .../queries/q28/gpu_bdb_query_28_dask_sql.py | 18 +++-- .../queries/q29/gpu_bdb_query_29_dask_sql.py | 22 +++--- .../queries/q30/gpu_bdb_query_30_dask_sql.py | 32 ++++----- 36 files changed, 434 insertions(+), 582 deletions(-) create mode 100644 gpu_bdb/queries/q20/dask-worker-space/global.lock create mode 100644 gpu_bdb/queries/q20/dask-worker-space/purge.lock diff --git a/gpu_bdb/bdb_tools/cluster_startup.py b/gpu_bdb/bdb_tools/cluster_startup.py index ec96625b..3c231483 100755 --- a/gpu_bdb/bdb_tools/cluster_startup.py +++ b/gpu_bdb/bdb_tools/cluster_startup.py @@ -24,43 +24,13 @@ from dask.utils import parse_bytes -def get_bsql_config_options(): - """Loads configuration environment variables. - In case it is not previously set, returns a default value for each one. - - Returns a dictionary object. - For more info: https://docs.blazingdb.com/docs/config_options - """ - config_options = {} - config_options['JOIN_PARTITION_SIZE_THRESHOLD'] = os.environ.get("JOIN_PARTITION_SIZE_THRESHOLD", 300000000) - config_options['MAX_DATA_LOAD_CONCAT_CACHE_BYTE_SIZE'] = os.environ.get("MAX_DATA_LOAD_CONCAT_CACHE_BYTE_SIZE", 400000000) - config_options['BLAZING_DEVICE_MEM_CONSUMPTION_THRESHOLD'] = os.environ.get("BLAZING_DEVICE_MEM_CONSUMPTION_THRESHOLD", 0.6) - config_options['BLAZ_HOST_MEM_CONSUMPTION_THRESHOLD'] = os.environ.get("BLAZ_HOST_MEM_CONSUMPTION_THRESHOLD", 0.6) - config_options['MAX_KERNEL_RUN_THREADS'] = os.environ.get("MAX_KERNEL_RUN_THREADS", 3) - config_options['TABLE_SCAN_KERNEL_NUM_THREADS'] = os.environ.get("TABLE_SCAN_KERNEL_NUM_THREADS", 1) - config_options['MAX_NUM_ORDER_BY_PARTITIONS_PER_NODE'] = os.environ.get("MAX_NUM_ORDER_BY_PARTITIONS_PER_NODE", 20) - config_options['NUM_BYTES_PER_ORDER_BY_PARTITION'] = os.environ.get("NUM_BYTES_PER_ORDER_BY_PARTITION", 400000000) - config_options['MAX_ORDER_BY_SAMPLES_PER_NODE'] = os.environ.get("MAX_ORDER_BY_SAMPLES_PER_NODE", 10000) - config_options['MAX_SEND_MESSAGE_THREADS'] = os.environ.get("MAX_SEND_MESSAGE_THREADS", 20) - config_options['MEMORY_MONITOR_PERIOD'] = os.environ.get("MEMORY_MONITOR_PERIOD", 50) - config_options['TRANSPORT_BUFFER_BYTE_SIZE'] = os.environ.get("TRANSPORT_BUFFER_BYTE_SIZE", 1048576) # 1 MBs - config_options['TRANSPORT_POOL_NUM_BUFFERS'] = os.environ.get("TRANSPORT_POOL_NUM_BUFFERS", 1000) - config_options['BLAZING_LOGGING_DIRECTORY'] = os.environ.get("BLAZING_LOGGING_DIRECTORY", 'blazing_log') - config_options['BLAZING_CACHE_DIRECTORY'] = os.environ.get("BLAZING_CACHE_DIRECTORY", '/tmp/') - config_options['LOGGING_LEVEL'] = os.environ.get("LOGGING_LEVEL", "trace") - config_options['MAX_JOIN_SCATTER_MEM_OVERHEAD'] = os.environ.get("MAX_JOIN_SCATTER_MEM_OVERHEAD", 500000000) - config_options['PROTOCOL'] = os.environ.get("PROTOCOL", "AUTO") - - return config_options - - -def attach_to_cluster(config, create_blazing_context=False): +def attach_to_cluster(config, create_sql_context=False): """Attaches to an existing cluster if available. By default, tries to attach to a cluster running on localhost:8786 (dask's default). This is currently hardcoded to assume the dashboard is running on port 8787. - Optionally, this will also create a BlazingContext. + Optionally, this will also create a Dask-SQL Context. """ scheduler_file = config.get("scheduler_file_path") host = config.get("cluster_host") @@ -131,14 +101,13 @@ def maybe_create_worker_directories(dask_worker): config["40GB_workers"] = worker_counts.get("40GB", 0) config["80GB_workers"] = worker_counts.get("80GB", 0) - bc = None - create_blazing_context = True - if create_blazing_context: - print('Creating context..') + c = None + create_sql_context = True + if create_sql_context: from dask_sql import Context - bc = Context() + c = Context() - return client, bc + return client, c def worker_count_info(client): diff --git a/gpu_bdb/bdb_tools/utils.py b/gpu_bdb/bdb_tools/utils.py index f807586a..e54d43c1 100755 --- a/gpu_bdb/bdb_tools/utils.py +++ b/gpu_bdb/bdb_tools/utils.py @@ -249,14 +249,14 @@ def remove_benchmark_files(): # Query Runner Utilities ################################# def run_query( - config, client, query_func, write_func=write_result, blazing_context=None + config, client, query_func, write_func=write_result, sql_context=None ): - if blazing_context: - run_bsql_query( + if sql_context: + run_sql_query( config=config, client=client, query_func=query_func, - blazing_context=blazing_context, + sql_context=sql_context, write_func=write_func, ) else: @@ -303,8 +303,8 @@ def run_dask_cudf_query(config, client, query_func, write_func=write_result): push_payload_to_googlesheet(config) -def run_bsql_query( - config, client, query_func, blazing_context, write_func=write_result +def run_sql_query( + config, client, query_func, sql_context, write_func=write_result ): """ Common utility to perform all steps needed to execute a dask-cudf version @@ -320,7 +320,7 @@ def run_bsql_query( dask_profile=config.get("dask_profile"), data_dir=data_dir, client=client, - bc=blazing_context, + c=sql_context, config=config, ) @@ -382,7 +382,7 @@ def gpubdb_argparser(): "sheet": os.environ.get("GOOGLE_SPREADSHEET_NAME"), "tab": os.environ.get("GOOGLE_SPREADSHEET_TAB"), "scheduler_file_path": os.environ.get("SCHEDULER_FILE"), - "benchmark_runner_include_bsql": os.environ.get("RUNNER_INCLUDE_BSQL"), + "benchmark_runner_include_sql": os.environ.get("RUNNER_INCLUDE_SQL"), } for key in args.keys(): @@ -789,7 +789,7 @@ def build_benchmark_googlesheet_payload(config): "Protocol": "UCX" if data.get("nvlink") == True else "TCP", "NVLINK": data.get("nvlink", "NA"), "Infiniband": data.get("infiniband", "NA"), - "Query Type": "blazing" if is_blazing_query() else "dask", + "Query Type": "sql" if is_sql_query() else "dask", "File Format": data.get("file_format"), "Time (seconds)": query_time + writing_time if query_time and writing_time @@ -810,7 +810,7 @@ def build_benchmark_googlesheet_payload(config): "Data Location": data.get("data_dir"), "Current Time": current_time, "cuDF Version": data.get("cudf"), - "BlazingSQL Version": data.get("blazingsql"), + "Dask SQL Version": data.get("sql"), "Dask Version": data.get("dask"), "Distributed Version": data.get("distributed"), "Dask-CUDA Version": data.get("dask-cuda"), @@ -827,15 +827,15 @@ def build_benchmark_googlesheet_payload(config): return payload -def is_blazing_query(): +def is_sql_query(): """ - Method that returns true if caller of the utility is a blazing query, returns false otherwise + Method that returns true if caller of the utility is a SQL query, returns false otherwise Assumes that caller is 3 levels above the stack - query_of_interest -> utils.push_to_google_sheet -> utils.build_payload -> utils.is_blazing_query + query_of_interest -> utils.push_to_google_sheet -> utils.build_payload -> utils.is_sql_query - Another potential solution is checking sys.modules.get("blazing") to check blazing is imported + Another potential solution is checking sys.modules.get("dask_sql") to check Dask-SQL is imported """ - return "bsql" in inspect.stack()[-3].function + return "sql" in inspect.stack()[-3].function def _get_benchmarked_method_time( @@ -866,7 +866,7 @@ def generate_library_information(): "dask-cuda", "rmm", "cupy", - "blazingsql", + "dask-sql", ] conda_list_command = ( @@ -958,6 +958,7 @@ def train_clustering_model(training_df, n_clusters, max_iter, n_init): random_state=np.random.randint(0, 500), init="k-means++", ) + print('fitting') model.fit(training_df) score = model.inertia_ @@ -969,6 +970,7 @@ def train_clustering_model(training_df, n_clusters, max_iter, n_init): elif abs(score) < abs(best_sse): best_sse = score best_model = model + print(f'done fitting: {i}') return { "cid_labels": best_model.labels_, diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index 0373034e..362ff1bd 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -5,7 +5,7 @@ import time import uuid -N_REPEATS = 1 +N_REPEATS = 5 def get_qnum_from_filename(name): @@ -23,8 +23,13 @@ def load_query(qnum, fn): dask_qnums = [str(i).zfill(2) for i in range(1, 31)] dask_qnums = [] -bsql_qnums = [str(i).zfill(2) for i in range(1, 31)] -bsql_qnums = ['01', '05', '08', '26', '27'] +sql_qnums = [str(i).zfill(2) for i in range(1, 31)] +#exclude = ['20'] +exclude = ['05'] +#exclude = ['05', '06', '07', '08', '09'] +for query in exclude: + sql_qnums = [q for q in sql_qnums if q != query] +print(sql_qnums) if __name__ == "__main__": @@ -34,50 +39,50 @@ def load_query(qnum, fn): import_query_libs() config = gpubdb_argparser() config["run_id"] = uuid.uuid4().hex - include_blazing = config.get("benchmark_runner_include_bsql") + include_sql = config.get("benchmark_runner_include_sql") dask_queries = { qnum: load_query(qnum, f"queries/q{qnum}/gpu_bdb_query_{qnum}.py") for qnum in dask_qnums } - include_blazing = True - if include_blazing: - bsql_queries = { + include_sql = True + if include_sql: + sql_queries = { qnum: load_query(qnum, f"queries/q{qnum}/gpu_bdb_query_{qnum}_dask_sql.py") - for qnum in bsql_qnums + for qnum in sql_qnums } - client, bc = attach_to_cluster(config, create_blazing_context=include_blazing) + client, c = attach_to_cluster(config, create_sql_context=include_sql) # Preload required libraries for queries on all workers client.run(import_query_libs) base_path = os.getcwd() - # Run BSQL Queries - if include_blazing and len(bsql_qnums) > 0: - print("Blazing Queries") - for qnum, q_func in bsql_queries.items(): - print(qnum) - - qpath = f"{base_path}/queries/q{qnum}/" - os.chdir(qpath) - if os.path.exists("current_query_num.txt"): - os.remove("current_query_num.txt") - with open("current_query_num.txt", "w") as fp: - fp.write(qnum) - - for r in range(N_REPEATS): - run_query( - config=config, - client=client, - query_func=q_func, - blazing_context=bc, - ) - client.run(gc.collect) - client.run_on_scheduler(gc.collect) - gc.collect() - time.sleep(3) + # Run Dask SQL Queries + if include_sql and len(sql_qnums) > 0: + print("Dask SQL Queries") + for r in range(N_REPEATS): + for qnum, q_func in sql_queries.items(): + print(f"{r}: {qnum}") + + qpath = f"{base_path}/queries/q{qnum}/" + os.chdir(qpath) + if os.path.exists("current_query_num.txt"): + os.remove("current_query_num.txt") + with open("current_query_num.txt", "w") as fp: + fp.write(qnum) + + run_query( + config=config, + client=client, + query_func=q_func, + sql_context=c, + ) + client.run(gc.collect) + client.run_on_scheduler(gc.collect) + gc.collect() + time.sleep(3) # Run Pure Dask Queries if len(dask_qnums) > 0: diff --git a/gpu_bdb/queries/load_test/gpu_bdb_load_test.py b/gpu_bdb/queries/load_test/gpu_bdb_load_test.py index ab57193f..830e9977 100755 --- a/gpu_bdb/queries/load_test/gpu_bdb_load_test.py +++ b/gpu_bdb/queries/load_test/gpu_bdb_load_test.py @@ -24,7 +24,7 @@ tables = [table.split(".")[0] for table in os.listdir(spark_schema_dir)] scale = [x for x in config["data_dir"].split("/") if "sf" in x][0] -part_size = 3 +part_size = 2 chunksize = "128 MiB" # Spark uses different names for column types, and RAPIDS doesn't yet support Decimal types. @@ -127,7 +127,7 @@ def repartition(table, outdir, npartitions=None, chunksize=None, compression="sn ) read_csv_table(table, chunksize).repartition( npartitions=npartitions - ).to_parquet(outdir + table, compression=compression) + ).to_parquet(outdir + table, compression=compression, index=False) def main(client, config): diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index a51447da..8e43fea3 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -1,6 +1,5 @@ # # Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -42,7 +41,7 @@ ss_cols = ["ss_item_sk", "ss_store_sk", "ss_ticket_number"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -52,15 +51,11 @@ def read_tables(data_dir, bc, config): item_df = table_reader.read("item", relevant_cols=item_cols) ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - bc.create_table("item", item_df, persist=False) - bc.create_table("store_sales", ss_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("store_sales", ss_df, persist=False) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_distinct = f""" SELECT DISTINCT ss_item_sk, ss_ticket_number @@ -69,11 +64,11 @@ def main(data_dir, client, bc, config): AND i.i_category_id IN ({q01_i_category_id_IN}) AND s.ss_store_sk IN ({q01_ss_store_sk_IN}) """ - result_distinct = bc.sql(query_distinct) + result_distinct = c.sql(query_distinct) result_distinct = result_distinct.persist() wait(result_distinct) - bc.create_table("distinct_table", result_distinct, persist=False) + c.create_table("distinct_table", result_distinct, persist=False) query = f""" SELECT item_sk_1, item_sk_2, COUNT(*) AS cnt @@ -92,13 +87,13 @@ def main(data_dir, client, bc, config): CAST(item_sk_2 AS VARCHAR) LIMIT {q01_limit} """ - result = bc.sql(query) + result = c.sql(query) - bc.drop_table("distinct_table") + c.drop_table("distinct_table") return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index 00ead45f..fae5aaa8 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -37,7 +37,7 @@ q02_session_timeout_inSec = 3600 -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -46,13 +46,11 @@ def read_tables(data_dir, bc, config): wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df, persist=False) - # bc.create_table("web_clickstreams", - # os.path.join(data_dir, "web_clickstreams/*.parquet")) + c.create_table("web_clickstreams", wcs_df, persist=False) -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT @@ -64,7 +62,7 @@ def main(data_dir, client, bc, config): AND wcs_user_sk IS NOT NULL DISTRIBUTE BY wcs_user_sk """ - wcs_result = bc.sql(query_1) + wcs_result = c.sql(query_1) session_df = wcs_result.map_partitions( get_distinct_sessions, @@ -73,7 +71,7 @@ def main(data_dir, client, bc, config): ) del wcs_result - bc.create_table('session_df', session_df, persist=False) + c.create_table('session_df', session_df, persist=False) last_query = f""" WITH item_df AS ( @@ -92,17 +90,17 @@ def main(data_dir, client, bc, config): ORDER BY cnt desc LIMIT {q02_limit} """ - result = bc.sql(last_query) + result = c.sql(last_query) result["item_sk_2"] = q02_item_sk result_order = ["item_sk_1", "item_sk_2", "cnt"] result = result[result_order] del session_df - bc.drop_table("session_df") + c.drop_table("session_df") return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 97dc12b7..12d3ebdd 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -141,7 +141,7 @@ def apply_find_items_viewed(df, item_mappings): return filtered -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -160,23 +160,23 @@ def read_tables(data_dir, bc, config): item_df = table_reader.read("item", relevant_cols=item_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df, persist=False) - bc.create_table("item", item_df, persist=False) + c.create_table("web_clickstreams", wcs_df, persist=False) + c.create_table("item", item_df, persist=False) -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT i_item_sk, CAST(i_category_id AS TINYINT) AS i_category_id FROM item """ - item_df = bc.sql(query_1) + item_df = c.sql(query_1) item_df = item_df.persist() wait(item_df) - bc.create_table("item_df", item_df, persist=False) + c.create_table("item_df", item_df, persist=False) query_2 = """ SELECT CAST(w.wcs_user_sk AS INTEGER) as wcs_user_sk, @@ -189,26 +189,26 @@ def main(data_dir, client, bc, config): AND w.wcs_item_sk IS NOT NULL DISTRIBUTE BY wcs_user_sk """ - merged_df = bc.sql(query_2) + merged_df = c.sql(query_2) query_3 = f""" SELECT i_item_sk, i_category_id FROM item_df WHERE i_category_id IN ({q03_purchased_item_category_IN}) """ - item_df_filtered = bc.sql(query_3) + item_df_filtered = c.sql(query_3) product_view_results = merged_df.map_partitions( apply_find_items_viewed, item_mappings=item_df_filtered ) - bc.drop_table("item_df") + c.drop_table("item_df") del item_df del merged_df del item_df_filtered - bc.create_table('product_result', product_view_results, persist=False) + c.create_table('product_result', product_view_results, persist=False) last_query = f""" SELECT CAST({q03_purchased_item_IN} AS BIGINT) AS purchased_item, @@ -219,14 +219,14 @@ def main(data_dir, client, bc, config): ORDER BY purchased_item, cnt desc, lastviewed_item LIMIT {q03_limit} """ - result = bc.sql(last_query) + result = c.sql(last_query) - bc.drop_table("product_result") + c.drop_table("product_result") del product_view_results return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index 794ff4c9..874b6a48 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -85,7 +85,7 @@ def reduction_function(df, keep_cols, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): return df -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -104,22 +104,18 @@ def read_tables(data_dir, bc, config): ] wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table('web_page_wo_categorical', wp_df, persist=False) - bc.create_table('web_clickstreams', wcs_df, persist=False) + c.create_table('web_page_wo_categorical', wp_df, persist=False) + c.create_table('web_clickstreams', wcs_df, persist=False) - # bc.create_table('web_page_wo_categorical', os.path.join(data_dir, "web_page/*.parquet")) - # bc.create_table('web_clickstreams', - # os.path.join(data_dir, "web_clickstreams/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_web_page = """ SELECT wp_type, wp_web_page_sk FROM web_page_wo_categorical """ - wp = bc.sql(query_web_page) + wp = c.sql(query_web_page) # Convert wp_type to categorical and get cat_id of review and dynamic type wp["wp_type"] = wp["wp_type"].map_partitions( @@ -143,7 +139,7 @@ def main(data_dir, client, bc, config): wp = wp.persist() wait(wp) - bc.create_table('web_page', wp, persist=False) + c.create_table('web_page', wp, persist=False) query = """ SELECT @@ -157,7 +153,7 @@ def main(data_dir, client, bc, config): AND c.wcs_sales_sk IS NULL --abandoned implies: no sale DISTRIBUTE BY wcs_user_sk """ - merged_df = bc.sql(query) + merged_df = c.sql(query) keep_cols = ["wcs_user_sk", "wp_type_codes", "tstamp_inSec"] result_df = merged_df.map_partitions( @@ -170,11 +166,11 @@ def main(data_dir, client, bc, config): result = result.compute() result_df = cudf.DataFrame({"sum(pagecount)/count(*)": [result]}) - bc.drop_table("web_page") + c.drop_table("web_page") return result_df if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index e3aca1d1..c7cd5c18 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -47,7 +47,7 @@ customer_columns = ["c_customer_sk", "c_current_cdemo_sk"] customer_dem_columns = ["cd_demo_sk", "cd_gender", "cd_education_status"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -65,17 +65,10 @@ def read_tables(data_dir, bc, config): "web_clickstreams", relevant_cols=wcs_columns, index=False ) - bc.create_table("web_clickstreams", wcs_ddf, persist=False) - bc.create_table("customer", customer_ddf, persist=False) - bc.create_table("item", item_ddf, persist=False) - bc.create_table("customer_demographics", customer_dem_ddf, persist=False) - - # bc.create_table("web_clickstreams", os.path.join(data_dir, "web_clickstreams/*.parquet")) - # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - # bc.create_table( - # "customer_demographics", os.path.join(data_dir, "customer_demographics/*.parquet" - # )) + c.create_table("web_clickstreams", wcs_ddf, persist=False) + c.create_table("customer", customer_ddf, persist=False) + c.create_table("item", item_ddf, persist=False) + c.create_table("customer_demographics", customer_dem_ddf, persist=False) def build_and_predict_model(ml_input_df): @@ -119,8 +112,8 @@ def build_and_predict_model(ml_input_df): return results_dict -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = """ SELECT @@ -160,7 +153,7 @@ def main(data_dir, client, bc, config): INNER JOIN customer_demographics ON c_current_cdemo_sk = cd_demo_sk """ - cust_and_clicks_ddf = bc.sql(query) + cust_and_clicks_ddf = c.sql(query) cust_and_clicks_ddf = cust_and_clicks_ddf.repartition(npartitions=1) @@ -184,5 +177,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index 2eeb1c5f..0feb99a8 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -35,7 +35,7 @@ q06_YEAR = 2001 -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -75,19 +75,14 @@ def read_tables(data_dir, bc, config): date_df = table_reader.read("date_dim", relevant_cols=date_cols) customer_df = table_reader.read("customer", relevant_cols=customer_cols) - bc.create_table('web_sales', ws_df, persist=False) - bc.create_table('store_sales', ss_df, persist=False) - bc.create_table('date_dim', date_df, persist=False) - bc.create_table('customer', customer_df, persist=False) + c.create_table('web_sales', ws_df, persist=False) + c.create_table('store_sales', ss_df, persist=False) + c.create_table('date_dim', date_df, persist=False) + c.create_table('customer', customer_df, persist=False) - # bc.create_table('web_sales', os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table('store_sales', os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table('customer', os.path.join(data_dir, "customer/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = f""" WITH temp_table_1 as @@ -150,12 +145,12 @@ def main(data_dir, client, bc, config): c_login LIMIT {q06_LIMIT} """ - result = bc.sql(query) + result = c.sql(query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index 56ef9525..e4e8af91 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -30,7 +30,7 @@ from bdb_tools.readers import build_reader -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -53,21 +53,15 @@ def read_tables(data_dir, bc, config): "customer_address", relevant_cols=customer_address_cols ) - bc.create_table("item", item_df, persist=False) - bc.create_table("customer", customer_df, persist=False) - bc.create_table("store_sales", store_sales_df, persist=False) - bc.create_table("date_dim", date_dim_df, persist=False) - bc.create_table("customer_address", customer_address_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("customer", customer_df, persist=False) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("customer_address", customer_address_df, persist=False) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table("customer_address", os.path.join(data_dir, "customer_address/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = """ WITH temp_table as @@ -106,11 +100,11 @@ def main(data_dir, client, bc, config): LIMIT 10 """ - result = bc.sql(query) + result = c.sql(query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 606569eb..08437fc2 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -144,7 +144,7 @@ def prep_for_sessionization(df, review_cat_code): return df_filtered -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -167,19 +167,14 @@ def read_tables(data_dir, bc, config): web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df, persist=False) - bc.create_table("web_sales", web_sales_df, persist=False) - bc.create_table("web_page", web_page_df, persist=False) - bc.create_table("date_dim", date_dim_df, persist=False) + c.create_table("web_clickstreams", wcs_df, persist=False) + c.create_table("web_sales", web_sales_df, persist=False) + c.create_table("web_page", web_page_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) - # bc.create_table("web_clickstreams", os.path.join(data_dir, "web_clickstreams/*.parquet")) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table("web_page", os.path.join(data_dir, "web_page/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = f""" SELECT d_date_sk @@ -188,7 +183,7 @@ def main(data_dir, client, bc, config): date '{q08_ENDDATE}') ORDER BY CAST(d_date as date) asc """ - result_dates_sk_filter = bc.sql(query_1).compute() + result_dates_sk_filter = c.sql(query_1).compute() # because `result_dates_sk_filter` has repetitive index result_dates_sk_filter.index = list(range(0, result_dates_sk_filter.shape[0])) @@ -201,7 +196,7 @@ def main(data_dir, client, bc, config): wp_type FROM web_page """ - web_page_df = bc.sql(query_aux) + web_page_df = c.sql(query_aux) # cast to minimum viable dtype web_page_df["wp_type"] = web_page_df["wp_type"].map_partitions( @@ -222,7 +217,7 @@ def main(data_dir, client, bc, config): web_page_df = web_page_df.persist() wait(web_page_df) - bc.create_table('web_page_2', web_page_df, persist=False) + c.create_table('web_page_2', web_page_df, persist=False) query_2 = f""" SELECT @@ -237,9 +232,9 @@ def main(data_dir, client, bc, config): --in the future we want to remove this ORDER BY DISTRIBUTE BY wcs_user_sk """ - merged_df = bc.sql(query_2) + merged_df = c.sql(query_2) - bc.drop_table("web_page_2") + c.drop_table("web_page_2") del web_page_df merged_df = merged_df.shuffle(on=["wcs_user_sk"]) @@ -259,7 +254,7 @@ def main(data_dir, client, bc, config): unique_review_sales = unique_review_sales.persist() wait(unique_review_sales) - bc.create_table("reviews", unique_review_sales, persist=False) + c.create_table("reviews", unique_review_sales, persist=False) last_query = f""" SELECT CAST(review_total AS BIGINT) AS q08_review_sales_amount, @@ -274,13 +269,13 @@ def main(data_dir, client, bc, config): WHERE ws_sold_date_sk between {q08_start_dt} AND {q08_end_dt} ) """ - result = bc.sql(last_query) + result = c.sql(last_query) - bc.drop_table("reviews") + c.drop_table("reviews") return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index 1c962df0..3fe702f6 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -60,7 +60,7 @@ q09_part3_sales_price_max = 200 -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -93,23 +93,15 @@ def read_tables(data_dir, bc, config): s_columns = ["s_store_sk"] store = table_reader.read("store", relevant_cols=s_columns) - bc.create_table("store_sales", store_sales, persist=False) - bc.create_table("customer_address", customer_address, persist=False) - bc.create_table("customer_demographics", customer_demographics, persist=False) - bc.create_table("date_dim", date_dim, persist=False) - bc.create_table("store", store, persist=False) + c.create_table("store_sales", store_sales, persist=False) + c.create_table("customer_address", customer_address, persist=False) + c.create_table("customer_demographics", customer_demographics, persist=False) + c.create_table("date_dim", date_dim, persist=False) + c.create_table("store", store, persist=False) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("customer_address", os.path.join(data_dir, "customer_address/*.parquet")) - # bc.create_table( - # "customer_demographics", os.path.join(data_dir, "customer_demographics/*.parquet" - # )) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = f""" SELECT SUM(ss1.ss_quantity) @@ -170,12 +162,12 @@ def main(data_dir, client, bc, config): ) ) """ - result = bc.sql(query) + result = c.sql(query) result.columns = ["sum(ss_quantity)"] return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index 8ff8a883..2f60a48e 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -41,7 +41,7 @@ eol_char = "รจ" -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): ### splitting by row groups for better parallelism table_reader = build_reader( @@ -55,13 +55,11 @@ def read_tables(data_dir, bc, config): "product_reviews", relevant_cols=product_reviews_cols, ) - bc.create_table("product_reviews", product_reviews_df, persist=False) + c.create_table("product_reviews", product_reviews_df, persist=False) - # bc.create_table('product_reviews', os.path.join(data_dir, "product_reviews/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT pr_item_sk, @@ -71,7 +69,7 @@ def main(data_dir, client, bc, config): where pr_review_content IS NOT NULL ORDER BY pr_item_sk, pr_review_content, pr_review_sk """ - product_reviews_df = bc.sql(query_1) + product_reviews_df = c.sql(query_1) product_reviews_df[ "pr_review_content" @@ -99,24 +97,24 @@ def main(data_dir, client, bc, config): product_reviews_df = product_reviews_df.persist() wait(product_reviews_df) - bc.create_table('product_reviews_df', product_reviews_df, persist=False) + c.create_table('product_reviews_df', product_reviews_df, persist=False) sentences = sentences.persist() wait(sentences) - bc.create_table('sentences', sentences, persist=False) + c.create_table('sentences', sentences, persist=False) # These files come from the official TPCx-BB kit # We extracted them from bigbenchqueriesmr.jar # Need to pass the absolute path for these txt files sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"], persist=False) - bc.create_table('negative_sentiment', ns_df, persist=False) + c.create_table('negative_sentiment', ns_df, persist=False) ps_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "positiveSentiment.txt"), names=["sentiment_word"], persist=False) - bc.create_table('positive_sentiment', ps_df, persist=False) + c.create_table('positive_sentiment', ps_df, persist=False) word_df = word_df.persist() wait(word_df) - bc.create_table('word_df', word_df, persist=False) + c.create_table('word_df', word_df, persist=False) query = ''' SELECT pr_item_sk as item_sk, @@ -150,13 +148,13 @@ def main(data_dir, client, bc, config): ON temp.review_idx_global_pos = product_reviews_df.pr_review_sk ORDER BY item_sk, review_sentence, sentiment, sentiment_word ''' - result = bc.sql(query) + result = c.sql(query) - bc.drop_table("product_reviews_df") + c.drop_table("product_reviews_df") del product_reviews_df - bc.drop_table("sentences") + c.drop_table("sentences") del sentences - bc.drop_table("word_df") + c.drop_table("word_df") del word_df return result @@ -164,5 +162,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index 344e20ce..531b0703 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -30,7 +30,7 @@ from bdb_tools.readers import build_reader -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -55,17 +55,13 @@ def read_tables(data_dir, bc, config): ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) date_df = table_reader.read("date_dim", relevant_cols=date_cols) - bc.create_table("web_sales", ws_df, persist=False) - bc.create_table("product_reviews", pr_df, persist=False) - bc.create_table("date_dim", date_df, persist=False) + c.create_table("web_sales", ws_df, persist=False) + c.create_table("product_reviews", pr_df, persist=False) + c.create_table("date_dim", date_df, persist=False) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = """ WITH p AS @@ -93,7 +89,7 @@ def main(data_dir, client, bc, config): FROM s INNER JOIN p ON p.pr_item_sk = s.ws_item_sk """ - result = bc.sql(query) + result = c.sql(query) sales_corr = result["x"].corr(result["y"]).compute() result_df = cudf.DataFrame([sales_corr]) result_df.columns = ["corr(CAST(reviews_count AS DOUBLE), avg_rating)"] @@ -102,5 +98,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index 5d27fd18..435a41cc 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -37,7 +37,7 @@ wcs_cols = ["wcs_user_sk", "wcs_click_date_sk", "wcs_item_sk", "wcs_sales_sk"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -48,19 +48,13 @@ def read_tables(data_dir, bc, config): store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table("web_clickstreams", wcs_df, persist=False) - bc.create_table("store_sales", store_sales_df, persist=False) - bc.create_table("item", item_df, persist=False) + c.create_table("web_clickstreams", wcs_df, persist=False) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("item", item_df, persist=False) - # bc.create_table("web_clickstreams", - # os.path.join(data_dir, "web_clickstreams/*.parquet")) - # bc.create_table("store_sales", - # os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = f""" SELECT DISTINCT wcs_user_sk @@ -90,11 +84,11 @@ def main(data_dir, client, bc, config): AND wcs_click_date_sk < ss_sold_date_sk ORDER BY wcs_user_sk """ - result = bc.sql(query) + result = c.sql(query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index e592345b..de031526 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -32,7 +32,7 @@ from dask.distributed import wait -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -51,19 +51,14 @@ def read_tables(data_dir, bc, config): w_sales_cols = ["ws_sold_date_sk", "ws_bill_customer_sk", "ws_net_paid"] web_sales_df = table_reader.read("web_sales", relevant_cols=w_sales_cols) - bc.create_table("date_dim", date_dim_df, persist=False) - bc.create_table("customer", customer_df, persist=False) - bc.create_table("store_sales", s_sales_df, persist=False) - bc.create_table("web_sales", web_sales_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("customer", customer_df, persist=False) + c.create_table("store_sales", s_sales_df, persist=False) + c.create_table("web_sales", web_sales_df, persist=False) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT @@ -80,11 +75,11 @@ def main(data_dir, client, bc, config): GROUP BY ss.ss_customer_sk HAVING sum( case when (d_year = 2001) THEN ss_net_paid ELSE 0.0 END) > 0.0 """ - temp_table1 = bc.sql(query_1) + temp_table1 = c.sql(query_1) temp_table1 = temp_table1.persist() wait(temp_table1) - bc.create_table("temp_table1", temp_table1, persist=False) + c.create_table("temp_table1", temp_table1, persist=False) query_2 = """ SELECT ws.ws_bill_customer_sk AS customer_sk, @@ -100,11 +95,11 @@ def main(data_dir, client, bc, config): GROUP BY ws.ws_bill_customer_sk HAVING sum( case when (d_year = 2001) THEN ws_net_paid ELSE 0.0 END) > 0.0 """ - temp_table2 = bc.sql(query_2) + temp_table2 = c.sql(query_2) temp_table2 = temp_table2.persist() wait(temp_table2) - bc.create_table("temp_table2", temp_table2, persist=False) + c.create_table("temp_table2", temp_table2, persist=False) query = """ SELECT CAST(c_customer_sk AS BIGINT) as c_customer_sk, @@ -124,14 +119,14 @@ def main(data_dir, client, bc, config): c_last_name LIMIT 100 """ - result = bc.sql(query) + result = c.sql(query) - bc.drop_table("temp_table1") - bc.drop_table("temp_table2") + c.drop_table("temp_table1") + c.drop_table("temp_table2") return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index 20907afc..2cc5c0bf 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -29,7 +29,7 @@ from bdb_tools.readers import build_reader -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -50,21 +50,14 @@ def read_tables(data_dir, bc, config): td_columns = ["t_time_sk", "t_hour"] time_dim = table_reader.read("time_dim", relevant_cols=td_columns) - bc.create_table("household_demographics", household_demographics, persist=False) - bc.create_table("web_page", web_page, persist=False) - bc.create_table("web_sales", web_sales, persist=False) - bc.create_table("time_dim", time_dim, persist=False) + c.create_table("household_demographics", household_demographics, persist=False) + c.create_table("web_page", web_page, persist=False) + c.create_table("web_sales", web_sales, persist=False) + c.create_table("time_dim", time_dim, persist=False) - # bc.create_table( - # "household_demographics", os.path.join(data_dir, "household_demographics/*.parquet" - # )) - # bc.create_table("web_page", os.path.join(data_dir, "web_page/*.parquet")) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table("time_dim", os.path.join(data_dir, "time_dim/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = """ SELECT CASE WHEN pmc > 0.0 THEN CAST (amc AS DOUBLE) / CAST (pmc AS DOUBLE) ELSE -1.0 END AS am_pm_ratio @@ -85,11 +78,11 @@ def main(data_dir, client, bc, config): ) sum_am_pm """ - result = bc.sql(query) + result = c.sql(query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index 58218a20..1bf8556e 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -41,7 +41,7 @@ item_cols = ["i_item_sk", "i_category_id"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -52,13 +52,13 @@ def read_tables(data_dir, bc, config): date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) item_df = table_reader.read("item", relevant_cols=item_cols) - bc.create_table("store_sales", store_sales_df, persist=False) - bc.create_table("date_dim", date_dim_df, persist=False) - bc.create_table("item", item_df, persist=False) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("item", item_df, persist=False) -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = f""" SELECT * @@ -90,11 +90,11 @@ def main(data_dir, client, bc, config): WHERE slope <= 0.0 ORDER BY cat """ - result = bc.sql(query) + result = c.sql(query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index fd466798..16162d8b 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -43,7 +43,7 @@ item_cols = ["i_item_sk", "i_item_id"] warehouse_cols = ["w_warehouse_sk", "w_state"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -56,21 +56,15 @@ def read_tables(data_dir, bc, config): item_df = table_reader.read("item", relevant_cols=item_cols) warehouse_df = table_reader.read("warehouse", relevant_cols=warehouse_cols) - bc.create_table("web_sales", web_sales_df, persist=False) - bc.create_table("web_returns", web_returns_df, persist=False) - bc.create_table("date_dim", date_dim_df, persist=False) - bc.create_table("item", item_df, persist=False) - bc.create_table("warehouse", warehouse_df, persist=False) + c.create_table("web_sales", web_sales_df, persist=False) + c.create_table("web_returns", web_returns_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("warehouse", warehouse_df, persist=False) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table("web_returns", os.path.join(data_dir, "web_returns/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - # bc.create_table("warehouse", os.path.join(data_dir, "warehouse/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) date = datetime.datetime(2001, 3, 16) start = (date + timedelta(days=-30)).strftime("%Y-%m-%d") @@ -84,7 +78,7 @@ def main(data_dir, client, bc, config): ORDER BY CAST(d_date as date) ASC """ - dates = bc.sql(date_query) + dates = c.sql(date_query) cpu_dates = dates["d_date_sk"].compute().to_pandas() cpu_dates.index = list(range(0, cpu_dates.shape[0])) @@ -126,11 +120,11 @@ def main(data_dir, client, bc, config): LIMIT 100 """ - result = bc.sql(last_query) + result = c.sql(last_query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 217961f7..39cd6dcf 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -51,7 +51,7 @@ customer_address_cols = ["ca_address_sk", "ca_gmt_offset"] promotion_cols = ["p_channel_email", "p_channel_dmail", "p_channel_tv", "p_promo_sk"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -68,25 +68,17 @@ def read_tables(data_dir, bc, config): ) promotion_df = table_reader.read("promotion", relevant_cols=promotion_cols) - bc.create_table("store_sales", store_sales_df, persist=False) - bc.create_table("item", item_df, persist=False) - bc.create_table("customer", customer_df, persist=False) - bc.create_table("store", store_df, persist=False) - bc.create_table("date_dim", date_dim_df, persist=False) - bc.create_table("customer_address", customer_address_df, persist=False) - bc.create_table("promotion", promotion_df, persist=False) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("customer", customer_df, persist=False) + c.create_table("store", store_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("customer_address", customer_address_df, persist=False) + c.create_table("promotion", promotion_df, persist=False) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - # bc.create_table("customer", os.path.join(data_dir, "customer/*.parquet")) - # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table("customer_address", os.path.join(data_dir, "customer_address/*.parquet")) - # bc.create_table("promotion", os.path.join(data_dir, "promotion/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_date = f""" select min(d_date_sk) as min_d_date_sk, @@ -95,7 +87,7 @@ def main(data_dir, client, bc, config): where d_year = {q17_year} and d_moy = {q17_month} """ - dates_result = bc.sql(query_date).compute() + dates_result = c.sql(query_date).compute() min_date_sk_val = dates_result["min_d_date_sk"][0] max_date_sk_val = dates_result["max_d_date_sk"][0] @@ -129,11 +121,11 @@ def main(data_dir, client, bc, config): ) sum_promotional -- we don't need a 'ON' join condition. result is just two numbers. """ - result = bc.sql(query) + result = c.sql(query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index 62780872..de689ef3 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -120,7 +120,7 @@ def find_relevant_reviews(df, targets, str_col_name="pr_review_content"): return combined -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -149,19 +149,14 @@ def read_tables(data_dir, bc, config): "product_reviews", relevant_cols=product_reviews_cols, ) - bc.create_table("store", store, persist=False) - bc.create_table("store_sales", store_sales, persist=False) - bc.create_table("date_dim", date_dim, persist=False) - bc.create_table("product_reviews", product_reviews, persist=False) + c.create_table("store", store, persist=False) + c.create_table("store_sales", store_sales, persist=False) + c.create_table("date_dim", date_dim, persist=False) + c.create_table("product_reviews", product_reviews, persist=False) - # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = f""" WITH temp_table1 AS @@ -202,7 +197,7 @@ def main(data_dir, client, bc, config): ) SELECT * FROM temp_table1 """ - stores_with_regression = bc.sql(query_1) + stores_with_regression = c.sql(query_1) query_2 = """ SELECT pr_review_date, @@ -212,7 +207,7 @@ def main(data_dir, client, bc, config): WHERE pr_review_content IS NOT NULL ORDER BY pr_review_date, pr_review_content, pr_review_sk """ - no_nulls = bc.sql(query_2) + no_nulls = c.sql(query_2) targets = ( stores_with_regression.s_store_name.str.lower() @@ -254,11 +249,11 @@ def main(data_dir, client, bc, config): stores_with_regression = stores_with_regression.persist() wait(stores_with_regression) - bc.create_table("stores_with_regression", stores_with_regression, persist=False) + c.create_table("stores_with_regression", stores_with_regression, persist=False) combined = combined.persist() wait(combined) - bc.create_table("combined", combined, persist=False) + c.create_table("combined", combined, persist=False) query_3 = """ SELECT store_ID, @@ -267,12 +262,12 @@ def main(data_dir, client, bc, config): FROM stores_with_regression INNER JOIN combined ON s_store_name = word """ - temp_table2 = bc.sql(query_3) + temp_table2 = c.sql(query_3) - bc.drop_table("stores_with_regression") + c.drop_table("stores_with_regression") del stores_with_regression - bc.drop_table("combined") + c.drop_table("combined") del combined # REAL QUERY @@ -293,19 +288,19 @@ def main(data_dir, client, bc, config): # Need to pass the absolute path for this txt file sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) - bc.create_table('sent_df', ns_df, persist=False) + c.create_table('sent_df', ns_df, persist=False) word_df = word_df.persist() wait(word_df) - bc.create_table("word_df", word_df, persist=False) + c.create_table("word_df", word_df, persist=False) sentences = sentences.persist() wait(sentences) - bc.create_table("sentences", sentences, persist=False) + c.create_table("sentences", sentences, persist=False) temp_table2 = temp_table2.persist() wait(temp_table2) - bc.create_table("temp_table2", temp_table2, persist=False) + c.create_table("temp_table2", temp_table2, persist=False) query_4 = """ WITH sentences_table AS @@ -343,18 +338,18 @@ def main(data_dir, client, bc, config): ON wsswsi.review_idx_global_pos = tt2.pr_review_sk ORDER BY s_name, r_date, r_sentence, sentiment_word """ - result = bc.sql(query_4) + result = c.sql(query_4) - bc.drop_table("word_df") + c.drop_table("word_df") del word_df - bc.drop_table("sentences") + c.drop_table("sentences") del sentences - bc.drop_table("temp_table2") + c.drop_table("temp_table2") del temp_table2 return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index 064e334a..536d40f8 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -44,7 +44,7 @@ eol_char = "รจ" -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -69,19 +69,14 @@ def read_tables(data_dir, bc, config): "product_reviews", relevant_cols=product_reviews_cols ) - bc.create_table('web_returns', web_returns_df, persist=False) - bc.create_table('date_dim', date_dim_df, persist=False) - bc.create_table('product_reviews', product_reviews_df, persist=False) - bc.create_table('store_returns', store_returns_df, persist=False) - - # bc.create_table('web_returns', os.path.join(data_dir, "web_returns/*.parquet")) - # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table('product_reviews', os.path.join(data_dir, "product_reviews/*.parquet")) - # bc.create_table('store_returns', os.path.join(data_dir, "store_returns/*.parquet")) + c.create_table('web_returns', web_returns_df, persist=False) + c.create_table('date_dim', date_dim_df, persist=False) + c.create_table('product_reviews', product_reviews_df, persist=False) + c.create_table('store_returns', store_returns_df, persist=False) -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = f""" WITH dateFilter AS @@ -127,7 +122,7 @@ def main(data_dir, client, bc, config): SELECT * FROM extract_sentiment ORDER BY pr_item_sk, pr_review_content, pr_review_sk """ - merged_df = bc.sql(query) + merged_df = c.sql(query) # second step -- Sentiment Word Extraction merged_df["pr_review_sk"] = merged_df["pr_review_sk"].astype("int32") @@ -152,19 +147,19 @@ def main(data_dir, client, bc, config): # Need to pass the absolute path for this txt file sentiment_dir = os.path.join(config["data_dir"], "sentiment_files") ns_df = dask_cudf.read_csv(os.path.join(sentiment_dir, "negativeSentiment.txt"), names=["sentiment_word"]) - bc.create_table('sent_df', ns_df, persist=False) + c.create_table('sent_df', ns_df, persist=False) sentences = sentences.persist() wait(sentences) - bc.create_table('sentences_df', sentences, persist=False) + c.create_table('sentences_df', sentences, persist=False) word_df = word_df.persist() wait(word_df) - bc.create_table('word_df', word_df, persist=False) + c.create_table('word_df', word_df, persist=False) merged_df = merged_df.persist() wait(merged_df) - bc.create_table('merged_df', merged_df, persist=False) + c.create_table('merged_df', merged_df, persist=False) query = """ WITH negativesent AS @@ -194,13 +189,13 @@ def main(data_dir, client, bc, config): INNER JOIN merged_df ON pr_review_sk = review_idx_global_pos ORDER BY pr_item_sk, review_sentence, sentiment_word """ - result = bc.sql(query) + result = c.sql(query) - bc.drop_table("sentences_df") + c.drop_table("sentences_df") del sentences - bc.drop_table("word_df") + c.drop_table("word_df") del word_df - bc.drop_table("merged_df") + c.drop_table("merged_df") del merged_df return result @@ -208,5 +203,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q20/dask-worker-space/global.lock b/gpu_bdb/queries/q20/dask-worker-space/global.lock new file mode 100644 index 00000000..e69de29b diff --git a/gpu_bdb/queries/q20/dask-worker-space/purge.lock b/gpu_bdb/queries/q20/dask-worker-space/purge.lock new file mode 100644 index 00000000..e69de29b diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index d7d79bc6..a996a00c 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -64,7 +64,7 @@ def get_clusters(client, ml_input_df, feature_cols): return results_dict -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -89,15 +89,12 @@ def read_tables(data_dir, bc, config): "store_returns", relevant_cols=store_returns_cols ) - bc.create_table("store_sales", store_sales_df, persist=False) - bc.create_table("store_returns", store_returns_df, persist=False) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("store_returns", store_returns_df, persist=False) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("store_returns", os.path.join(data_dir, "store_returns/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = """ SELECT @@ -142,7 +139,7 @@ def main(data_dir, client, bc, config): GROUP BY sr_customer_sk ) returned ON ss_customer_sk=sr_customer_sk """ - final_df = bc.sql(query) + final_df = c.sql(query) final_df = final_df.fillna(0) final_df = final_df.repartition(npartitions=1).persist() @@ -163,5 +160,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index 30aeea4e..889ad9a7 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -49,7 +49,7 @@ store_cols = ["s_store_name", "s_store_id", "s_store_sk"] item_cols = ["i_item_id", "i_item_desc", "i_item_sk"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -63,23 +63,16 @@ def read_tables(data_dir, bc, config): store_table_df = table_reader.read("store", relevant_cols=store_cols) item_table_df = table_reader.read("item", relevant_cols=item_cols) - bc.create_table("store_sales", store_sales_df, persist=False) - bc.create_table("date_dim", date_dim_df, persist=False) - bc.create_table("item", item_table_df, persist=False) - bc.create_table("web_sales", web_sales_df, persist=False) - bc.create_table("store_returns", store_returns_df, persist=False) - bc.create_table("store", store_table_df, persist=False) - - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("date_dim", os.path.join(data_dir, "date_dim/*.parquet")) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table("store_returns", os.path.join(data_dir, "store_returns/*.parquet")) - # bc.create_table("store", os.path.join(data_dir, "store/*.parquet")) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("item", item_table_df, persist=False) + c.create_table("web_sales", web_sales_df, persist=False) + c.create_table("store_returns", store_returns_df, persist=False) + c.create_table("store", store_table_df, persist=False) -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = """ SELECT @@ -160,12 +153,12 @@ def main(data_dir, client, bc, config): part_s.s_store_name LIMIT 100 """ - result = bc.sql(query) + result = c.sql(query) result['i_item_desc'] = result['i_item_desc'].str.strip() return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index 83be2d11..bb780a7c 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -37,7 +37,7 @@ q22_i_current_price_max = "1.5" -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -61,19 +61,14 @@ def read_tables(data_dir, bc, config): date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) date_dim = date_dim.map_partitions(convert_datestring_to_days) - bc.create_table('inventory', inventory, persist=False) - bc.create_table('item', item, persist=False) - bc.create_table('warehouse', warehouse, persist=False) - bc.create_table('date_dim', date_dim, persist=False) + c.create_table('inventory', inventory, persist=False) + c.create_table('item', item, persist=False) + c.create_table('warehouse', warehouse, persist=False) + c.create_table('date_dim', date_dim, persist=False) - # bc.create_table('inventory', os.path.join(data_dir, "inventory/*.parquet")) - # bc.create_table('item', os.path.join(data_dir, "item/*.parquet")) - # bc.create_table('warehouse', os.path.join(data_dir, "warehouse/*.parquet")) - # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) # Filter limit in days min_date = np.datetime64(q22_date, "D").astype(int) - 30 @@ -100,8 +95,8 @@ def main(data_dir, client, bc, config): AND d_date <= {max_date} GROUP BY w_warehouse_name, i_item_id """ - intermediate = bc.sql(query) - bc.create_table("intermediate", intermediate ,persist=False) + intermediate = c.sql(query) + c.create_table("intermediate", intermediate ,persist=False) query_2 = f""" SELECT @@ -116,11 +111,11 @@ def main(data_dir, client, bc, config): ORDER BY w_warehouse_name, i_item_id LIMIT 100 """ - result = bc.sql(query_2) + result = c.sql(query_2) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index a25c7b3a..b12b433d 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -37,7 +37,7 @@ q23_coefficient = 1.3 -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -53,15 +53,12 @@ def read_tables(data_dir, bc, config): ] inv_df = table_reader.read("inventory", relevant_cols=inv_cols) - bc.create_table('inventory', inv_df, persist=False) - bc.create_table('date_dim', date_df, persist=False) + c.create_table('inventory', inv_df, persist=False) + c.create_table('date_dim', date_df, persist=False) - # bc.create_table('inventory', os.path.join(data_dir, "inventory/*.parquet")) - # bc.create_table('date_dim', os.path.join(data_dir, "date_dim/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = f""" SELECT inv_warehouse_sk, @@ -73,9 +70,9 @@ def main(data_dir, client, bc, config): AND d.d_year = {q23_year} AND d_moy between {q23_month} AND {q23_month + 1} """ - inv_dates_result = bc.sql(query_1) + inv_dates_result = c.sql(query_1) - bc.create_table('inv_dates', inv_dates_result, persist=False) + c.create_table('inv_dates', inv_dates_result, persist=False) query_2 = """ SELECT inv_warehouse_sk, inv_item_sk, @@ -85,9 +82,9 @@ def main(data_dir, client, bc, config): FROM inv_dates GROUP BY inv_warehouse_sk, inv_item_sk, d_moy """ - iteration_1 = bc.sql(query_2) + iteration_1 = c.sql(query_2) - bc.create_table('iteration_1', iteration_1, persist=False) + c.create_table('iteration_1', iteration_1, persist=False) query_3 = f""" SELECT inv_warehouse_sk, inv_item_sk, @@ -97,9 +94,9 @@ def main(data_dir, client, bc, config): WHERE (q_std / q_mean) >= {q23_coefficient} """ - iteration_2 = bc.sql(query_3) + iteration_2 = c.sql(query_3) - bc.create_table('temp_table', iteration_2, persist=False) + c.create_table('temp_table', iteration_2, persist=False) query = f""" SELECT inv1.inv_warehouse_sk, inv1.inv_item_sk, @@ -115,14 +112,14 @@ def main(data_dir, client, bc, config): ORDER BY inv1.inv_warehouse_sk, inv1.inv_item_sk """ - result = bc.sql(query) + result = c.sql(query) result = result.persist() wait(result) - bc.drop_table("temp_table") + c.drop_table("temp_table") return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index 81b6a675..90f5b681 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -40,7 +40,7 @@ ] ss_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_quantity"] -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -52,19 +52,14 @@ def read_tables(data_dir, bc, config): imp_df = table_reader.read("item_marketprices", relevant_cols=imp_cols) ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - bc.create_table("web_sales", ws_df, persist=False) - bc.create_table("item", item_df, persist=False) - bc.create_table("item_marketprices", imp_df, persist=False) - bc.create_table("store_sales", ss_df, persist=False) + c.create_table("web_sales", ws_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("item_marketprices", imp_df, persist=False) + c.create_table("store_sales", ss_df, persist=False) - # bc.create_table("web_sales", os.path.join(data_dir, "web_sales/*.parquet")) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - # bc.create_table("item_marketprices", os.path.join(data_dir, "item_marketprices/*.parquet")) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = """ WITH temp_table as @@ -111,11 +106,11 @@ def main(data_dir, client, bc, config): GROUP BY ws.ws_item_sk """ - result = bc.sql(query) + result = c.sql(query) return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index 7fe53d8a..2ed93722 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -61,7 +61,7 @@ def get_clusters(client, ml_input_df): return results_dict -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -81,9 +81,9 @@ def read_tables(data_dir, bc, config): ws_ddf = table_reader.read("web_sales", relevant_cols=ws_cols, index=False) datedim_ddf = table_reader.read("date_dim", relevant_cols=datedim_cols, index=False) - bc.create_table("web_sales", ws_ddf, persist=False) - bc.create_table("store_sales", ss_ddf, persist=False) - bc.create_table("date_dim", datedim_ddf, persist=False) + c.create_table("web_sales", ws_ddf, persist=False) + c.create_table("store_sales", ss_ddf, persist=False) + c.create_table("date_dim", datedim_ddf, persist=False) def agg_count_distinct(df, group_key, counted_key): @@ -102,8 +102,8 @@ def agg_count_distinct(df, group_key, counted_key): unique_df = unique_df.groupby(group_key)[counted_key].count() return unique_df.reset_index(drop=False) -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) q25_date = "2002-01-02" ss_join_query= f""" @@ -139,11 +139,11 @@ def main(data_dir, client, bc, config): ws_bill_customer_sk IS NOT NULL """ - ss_merged_df = bc.sql(ss_join_query) - ws_merged_df = bc.sql(ws_join_query) + ss_merged_df = c.sql(ss_join_query) + ws_merged_df = c.sql(ws_join_query) - bc.create_table('ss_merged_table', ss_merged_df, persist=False) - bc.create_table('ws_merged_table', ws_merged_df, persist=False) + c.create_table('ss_merged_table', ss_merged_df, persist=False) + c.create_table('ws_merged_table', ws_merged_df, persist=False) ss_agg_query = """ SELECT @@ -167,19 +167,19 @@ def main(data_dir, client, bc, config): ss_distinct_count_agg = agg_count_distinct(ss_merged_df,'ss_customer_sk','ss_ticket_number') ss_distinct_count_agg = ss_distinct_count_agg.rename(columns={'ss_customer_sk':'cid', 'ss_ticket_number':'frequency'}) - ss_agg_df = bc.sql(ss_agg_query) + ss_agg_df = c.sql(ss_agg_query) ### add distinct count ss_agg_df = ss_agg_df.merge(ss_distinct_count_agg) ws_distinct_count_agg = agg_count_distinct(ws_merged_df,'ws_bill_customer_sk','ws_order_number') ws_distinct_count_agg = ws_distinct_count_agg.rename(columns={'ws_bill_customer_sk':'cid', 'ws_order_number':'frequency'}) - ws_agg_df = bc.sql(ws_agg_query) + ws_agg_df = c.sql(ws_agg_query) ### add distinct count ws_agg_df = ws_agg_df.merge(ws_distinct_count_agg) - bc.create_table('ss_agg_df', ss_agg_df, persist=False) - bc.create_table('ws_agg_df', ws_agg_df, persist=False) + c.create_table('ss_agg_df', ss_agg_df, persist=False) + c.create_table('ws_agg_df', ws_agg_df, persist=False) result_query = ''' @@ -199,7 +199,7 @@ def main(data_dir, client, bc, config): GROUP BY cid ORDER BY cid ''' - cluster_input_ddf = bc.sql(result_query) + cluster_input_ddf = c.sql(result_query) # Prepare df for KMeans clustering cluster_input_ddf["recency"] = cluster_input_ddf["recency"].astype("int64") @@ -214,5 +214,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 70c8c9b1..52772aff 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -63,7 +63,7 @@ def get_clusters(client, kmeans_input_df): return results_dict -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -76,15 +76,12 @@ def read_tables(data_dir, bc, config): ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) - bc.create_table("store_sales", ss_ddf, persist=False) - bc.create_table("item", items_ddf, persist=False) + c.create_table("store_sales", ss_ddf, persist=False) + c.create_table("item", items_ddf, persist=False) - # bc.create_table("store_sales", os.path.join(data_dir, "store_sales/*.parquet")) - # bc.create_table("item", os.path.join(data_dir, "item/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query = f""" SELECT @@ -116,7 +113,7 @@ def main(data_dir, client, bc, config): HAVING count(ss.ss_item_sk) > {q26_count_ss_item_sk} ORDER BY cid """ - result = bc.sql(query) + result = c.sql(query) result = result.repartition(npartitions=1) result_ml = result.set_index('cid') ml_result_dict = get_clusters(client=client, kmeans_input_df=result_ml) @@ -126,4 +123,4 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() client, c = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=c) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index b709cf2d..00372b48 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -43,7 +43,7 @@ EOL_CHAR = "." -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): ### splitting by row groups for better parallelism table_reader = build_reader( data_format=config["file_format"], @@ -55,9 +55,7 @@ def read_tables(data_dir, bc, config): "product_reviews", relevant_cols=product_reviews_cols ) - bc.create_table("product_reviews", product_reviews_df, persist=False) - - # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) + c.create_table("product_reviews", product_reviews_df, persist=False) def ner_parser(df, col_string, batch_size=256): @@ -74,8 +72,8 @@ def ner_parser(df, col_string, batch_size=256): return df -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) import dask_cudf @@ -84,7 +82,7 @@ def main(data_dir, client, bc, config): FROM product_reviews WHERE pr_item_sk = {q27_pr_item_sk} """ - product_reviews_df = bc.sql(query) + product_reviews_df = c.sql(query) sentences = product_reviews_df.map_partitions( create_sentences_from_reviews, @@ -119,11 +117,11 @@ def main(data_dir, client, bc, config): # recombine repeated_names = repeated_names.persist() wait(repeated_names) - bc.create_table('repeated_names', repeated_names, persist=False) + c.create_table('repeated_names', repeated_names, persist=False) ner_parsed = ner_parsed.persist() wait(ner_parsed) - bc.create_table('ner_parsed', ner_parsed, persist=False) + c.create_table('ner_parsed', ner_parsed, persist=False) query = f""" SELECT review_idx_global_pos as review_sk, @@ -134,10 +132,10 @@ def main(data_dir, client, bc, config): ON sentence_idx_global_pos = sentence_tokenized_global_pos ORDER BY review_idx_global_pos, item_sk, word, sentence """ - recombined = bc.sql(query) + recombined = c.sql(query) - bc.drop_table("repeated_names") - bc.drop_table("ner_parsed") + c.drop_table("repeated_names") + c.drop_table("ner_parsed") del ner_parsed del repeated_names return recombined @@ -145,6 +143,6 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 681ee2f4..c77b2e00 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -299,7 +299,7 @@ def post_etl_processing(client, train_data, test_data): return final_data, acc, prec, cmat -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): ### splitting by row groups for better parallelism table_reader = build_reader( data_format=config["file_format"], @@ -314,13 +314,11 @@ def read_tables(data_dir, bc, config): ] pr_df = table_reader.read("product_reviews", relevant_cols=columns) - bc.create_table("product_reviews", pr_df, persist=False) + c.create_table("product_reviews", pr_df, persist=False) - # bc.create_table("product_reviews", os.path.join(data_dir, "product_reviews/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) # 10 % of data query1 = """ @@ -333,7 +331,7 @@ def main(data_dir, client, bc, config): AND pr_review_content IS NOT NULL ORDER BY pr_review_sk """ - test_data = bc.sql(query1) + test_data = c.sql(query1) # 90 % of data query2 = """ @@ -346,7 +344,7 @@ def main(data_dir, client, bc, config): AND pr_review_content IS NOT NULL ORDER BY pr_review_sk """ - train_data = bc.sql(query2) + train_data = c.sql(query2) final_data, acc, prec, cmat = post_etl_processing( client=client, train_data=train_data, test_data=test_data @@ -365,5 +363,5 @@ def main(data_dir, client, bc, config): if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index a2bf7eb6..d8dcf214 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -35,7 +35,7 @@ q29_limit = 100 -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], ) @@ -45,12 +45,12 @@ def read_tables(data_dir, bc, config): ws_cols = ["ws_order_number", "ws_item_sk"] ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) - bc.create_table('item', item_df, persist=False) - bc.create_table('web_sales', ws_df, persist=False) + c.create_table('item', item_df, persist=False) + c.create_table('web_sales', ws_df, persist=False) -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) n_workers = len(client.scheduler_info()["workers"]) join_query = """ @@ -65,7 +65,7 @@ def main(data_dir, client, bc, config): WHERE ws.ws_item_sk = i.i_item_sk AND i.i_category_id IS NOT NULL """ - result = bc.sql(join_query) + result = c.sql(join_query) # Distinct Calculatiin result_distinct = result.drop_duplicates(split_out=n_workers,ignore_index=True) @@ -73,7 +73,7 @@ def main(data_dir, client, bc, config): ## TODO Raise a issue for this result_distinct = result_distinct.reset_index(drop=True) ### Persiting cause Order by causes execution - bc.create_table('distinct_table', result_distinct, persist=True) + c.create_table('distinct_table', result_distinct, persist=True) query = f""" SELECT category_id_1, category_id_2, COUNT (*) AS cnt @@ -90,15 +90,15 @@ def main(data_dir, client, bc, config): ORDER BY cnt DESC, category_id_1, category_id_2 LIMIT {q29_limit} """ - result = bc.sql(query) + result = c.sql(query) result = result.persist() wait(result); - bc.drop_table("distinct_table") + c.drop_table("distinct_table") return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 816f5cac..9d298f5a 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -44,7 +44,7 @@ -def read_tables(data_dir, bc, config): +def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], @@ -57,27 +57,23 @@ def read_tables(data_dir, bc, config): wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - bc.create_table('web_clickstreams', wcs_df, persist=False) - bc.create_table('item', item_df, persist=False) - # print(len(wcs_df)) + c.create_table('web_clickstreams', wcs_df, persist=False) + c.create_table('item', item_df, persist=False) - # bc.create_table('web_clickstreams', os.path.join(data_dir, "web_clickstreams/*.parquet")) - # bc.create_table('item', os.path.join(data_dir, "item/*.parquet")) - -def main(data_dir, client, bc, config): - benchmark(read_tables, data_dir, bc, config, dask_profile=config["dask_profile"]) +def main(data_dir, client, c, config): + benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) query_1 = """ SELECT i_item_sk, CAST(i_category_id AS TINYINT) AS i_category_id FROM item """ - item_df = bc.sql(query_1) + item_df = c.sql(query_1) item_df = item_df.persist() wait(item_df) - bc.create_table("item_df", item_df, persist=False) + c.create_table("item_df", item_df, persist=False) # print(len(item_df)) # print(len(item_df.columns)) @@ -91,9 +87,9 @@ def main(data_dir, client, bc, config): AND wcs.wcs_user_sk IS NOT NULL DISTRIBUTE BY wcs_user_sk """ - merged_df = bc.sql(query_2) + merged_df = c.sql(query_2) - bc.drop_table("item_df") + c.drop_table("item_df") del item_df distinct_session_df = merged_df.map_partitions(get_distinct_sessions, @@ -108,7 +104,7 @@ def main(data_dir, client, bc, config): output_col_2="category_id_2") del distinct_session_df - bc.create_table('pair_df', pair_df, persist=False) + c.create_table('pair_df', pair_df, persist=False) last_query = f""" SELECT CAST(category_id_1 AS BIGINT) AS category_id_1, @@ -119,13 +115,13 @@ def main(data_dir, client, bc, config): ORDER BY cnt desc LIMIT {q30_limit} """ - result = bc.sql(last_query) + result = c.sql(last_query) - bc.drop_table("pair_df") + c.drop_table("pair_df") return result if __name__ == "__main__": config = gpubdb_argparser() - client, bc = attach_to_cluster(config) - run_query(config=config, client=client, query_func=main, blazing_context=bc) + client, c = attach_to_cluster(config) + run_query(config=config, client=client, query_func=main, sql_context=c) From d3ca1b11c927bd1471a59862dc616d149376bf6a Mon Sep 17 00:00:00 2001 From: sft-managed Date: Tue, 4 Jan 2022 12:56:57 -0800 Subject: [PATCH 29/51] Update copyrights --- gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py | 2 +- gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 3 +-- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 3 +-- gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 3 +-- gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py | 3 +-- gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py | 3 +-- gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py | 3 +-- gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 3 +-- gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py | 3 +-- gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py | 3 +-- gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py | 3 +-- gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py | 3 +-- gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py | 3 +-- gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py | 3 +-- gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py | 3 +-- gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py | 3 +-- gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py | 3 +-- gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py | 3 +-- gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py | 3 +-- gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py | 3 +-- gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py | 3 +-- gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py | 3 +-- gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py | 3 +-- gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py | 3 +-- gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py | 3 +-- gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py | 3 +-- gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py | 3 +-- gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py | 3 +-- gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 3 +-- gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 3 +-- 30 files changed, 30 insertions(+), 59 deletions(-) diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index 8e43fea3..9c9c0d2b 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index fae5aaa8..e2e9cb90 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 12d3ebdd..e0ed7669 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index 874b6a48..f2aabfae 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index c7cd5c18..7ccf34e3 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index 0feb99a8..e81c8ff3 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index e4e8af91..47ccab2f 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 08437fc2..9afa57b9 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index 3fe702f6..8f7355a1 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index 2f60a48e..e00022a5 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index 531b0703..2efd1514 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index 435a41cc..3db7d4ce 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index de031526..68d14d44 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index 2cc5c0bf..d34a367d 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index 1bf8556e..19d855b0 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 16162d8b..9a78c428 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 39cd6dcf..ccbe050b 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index de689ef3..8176e9c7 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index 536d40f8..0a767053 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index a996a00c..68a87fe1 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index 889ad9a7..bfe53354 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index bb780a7c..43b4a9bf 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index b12b433d..b97114d4 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index 90f5b681..07feaf87 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index 2ed93722..74443bb0 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 52772aff..09adcff0 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index 00372b48..eddc82c1 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index c77b2e00..787d00d8 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index d8dcf214..85a0d629 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 9d298f5a..aa3db4db 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -1,6 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. -# Copyright (c) 2019-2020, BlazingSQL, Inc. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. From f5303bb062eb6e5c0970399ce1d042bbf5d45bca Mon Sep 17 00:00:00 2001 From: sft-managed Date: Wed, 5 Jan 2022 15:24:43 +0000 Subject: [PATCH 30/51] Cleanup --- gpu_bdb/bdb_tools/utils.py | 2 -- gpu_bdb/benchmark_runner.py | 5 +---- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/gpu_bdb/bdb_tools/utils.py b/gpu_bdb/bdb_tools/utils.py index e54d43c1..a8183ef8 100755 --- a/gpu_bdb/bdb_tools/utils.py +++ b/gpu_bdb/bdb_tools/utils.py @@ -958,7 +958,6 @@ def train_clustering_model(training_df, n_clusters, max_iter, n_init): random_state=np.random.randint(0, 500), init="k-means++", ) - print('fitting') model.fit(training_df) score = model.inertia_ @@ -970,7 +969,6 @@ def train_clustering_model(training_df, n_clusters, max_iter, n_init): elif abs(score) < abs(best_sse): best_sse = score best_model = model - print(f'done fitting: {i}') return { "cid_labels": best_model.labels_, diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index 362ff1bd..dc59ebee 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -22,11 +22,8 @@ def load_query(qnum, fn): dask_qnums = [str(i).zfill(2) for i in range(1, 31)] -dask_qnums = [] sql_qnums = [str(i).zfill(2) for i in range(1, 31)] -#exclude = ['20'] -exclude = ['05'] -#exclude = ['05', '06', '07', '08', '09'] + for query in exclude: sql_qnums = [q for q in sql_qnums if q != query] print(sql_qnums) From 19a271dcacd404f4956b5eec5c08d6c6002b4ce3 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Thu, 6 Jan 2022 03:22:12 +0000 Subject: [PATCH 31/51] Address reviews --- gpu_bdb/bdb_tools/cluster_startup.py | 7 - gpu_bdb/bdb_tools/q03_utils.py | 128 ++++++++++++++++++ gpu_bdb/bdb_tools/q04_utils.py | 67 +++++++++ gpu_bdb/benchmark_runner.py | 37 +++-- .../benchmark_runner/benchmark_config.yaml | 2 +- .../queries/q01/gpu_bdb_query_01_dask_sql.py | 2 +- .../queries/q02/gpu_bdb_query_02_dask_sql.py | 2 +- gpu_bdb/queries/q03/gpu_bdb_query_03.py | 93 +------------ .../queries/q03/gpu_bdb_query_03_dask_sql.py | 110 +-------------- gpu_bdb/queries/q04/gpu_bdb_query_04.py | 61 +-------- .../queries/q04/gpu_bdb_query_04_dask_sql.py | 65 +-------- .../queries/q05/gpu_bdb_query_05_dask_sql.py | 2 +- .../queries/q06/gpu_bdb_query_06_dask_sql.py | 2 +- .../queries/q07/gpu_bdb_query_07_dask_sql.py | 2 +- .../queries/q08/gpu_bdb_query_08_dask_sql.py | 2 +- .../queries/q09/gpu_bdb_query_09_dask_sql.py | 2 +- .../queries/q10/gpu_bdb_query_10_dask_sql.py | 2 +- .../queries/q11/gpu_bdb_query_11_dask_sql.py | 2 +- .../queries/q12/gpu_bdb_query_12_dask_sql.py | 2 +- .../queries/q13/gpu_bdb_query_13_dask_sql.py | 2 +- .../queries/q14/gpu_bdb_query_14_dask_sql.py | 2 +- .../queries/q15/gpu_bdb_query_15_dask_sql.py | 2 +- .../queries/q16/gpu_bdb_query_16_dask_sql.py | 2 +- .../queries/q17/gpu_bdb_query_17_dask_sql.py | 2 +- .../queries/q18/gpu_bdb_query_18_dask_sql.py | 2 +- .../queries/q19/gpu_bdb_query_19_dask_sql.py | 2 +- .../queries/q20/gpu_bdb_query_20_dask_sql.py | 2 +- .../queries/q21/gpu_bdb_query_21_dask_sql.py | 2 +- .../queries/q22/gpu_bdb_query_22_dask_sql.py | 2 +- .../queries/q23/gpu_bdb_query_23_dask_sql.py | 2 +- .../queries/q24/gpu_bdb_query_24_dask_sql.py | 2 +- .../queries/q25/gpu_bdb_query_25_dask_sql.py | 2 +- .../queries/q26/gpu_bdb_query_26_dask_sql.py | 2 +- .../queries/q27/gpu_bdb_query_27_dask_sql.py | 2 +- .../queries/q28/gpu_bdb_query_28_dask_sql.py | 2 +- .../queries/q29/gpu_bdb_query_29_dask_sql.py | 2 +- .../queries/q30/gpu_bdb_query_30_dask_sql.py | 4 +- 37 files changed, 262 insertions(+), 366 deletions(-) create mode 100644 gpu_bdb/bdb_tools/q03_utils.py create mode 100644 gpu_bdb/bdb_tools/q04_utils.py diff --git a/gpu_bdb/bdb_tools/cluster_startup.py b/gpu_bdb/bdb_tools/cluster_startup.py index 3c231483..02e56640 100755 --- a/gpu_bdb/bdb_tools/cluster_startup.py +++ b/gpu_bdb/bdb_tools/cluster_startup.py @@ -102,7 +102,6 @@ def maybe_create_worker_directories(dask_worker): config["80GB_workers"] = worker_counts.get("80GB", 0) c = None - create_sql_context = True if create_sql_context: from dask_sql import Context c = Context() @@ -160,11 +159,5 @@ def import_query_libs(): "spacy", ] - # optionally include blazingsql - # this is brittle, but it resolves breaking change - # issues as we can control the environment - if os.environ.get("RUNNER_INCLUDE_BSQL"): - library_list.append("blazingsql") - for lib in library_list: importlib.import_module(lib) diff --git a/gpu_bdb/bdb_tools/q03_utils.py b/gpu_bdb/bdb_tools/q03_utils.py new file mode 100644 index 00000000..1d02e6d6 --- /dev/null +++ b/gpu_bdb/bdb_tools/q03_utils.py @@ -0,0 +1,128 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os +import sys + +import cudf + +from numba import cuda + +# -------- Q03 ----------- +q03_days_in_sec_before_purchase = 864000 +q03_views_before_purchase = 5 +q03_purchased_item_IN = 10001 + +@cuda.jit +def find_items_viewed_before_purchase_kernel( + relevant_idx_col, user_col, timestamp_col, item_col, out_col, N +): + """ + Find the past N items viewed after a relevant purchase was made, + as defined by the configuration of this query. + """ + i = cuda.grid(1) + relevant_item = q03_purchased_item_IN + + if i < (relevant_idx_col.size): # boundary guard + # every relevant row gets N rows in the output, so we need to map the indexes + # back into their position in the original array + orig_idx = relevant_idx_col[i] + current_user = user_col[orig_idx] + + # look at the previous N clicks (assume sorted descending) + rows_to_check = N + remaining_rows = user_col.size - orig_idx + + if remaining_rows <= rows_to_check: + rows_to_check = remaining_rows - 1 + + for k in range(1, rows_to_check + 1): + if current_user != user_col[orig_idx + k]: + out_col[i * N + k - 1] = 0 + + # only checking relevant purchases via the relevant_idx_col + elif (timestamp_col[orig_idx + k] <= timestamp_col[orig_idx]) & ( + timestamp_col[orig_idx + k] + >= (timestamp_col[orig_idx] - q03_days_in_sec_before_purchase) + ): + out_col[i * N + k - 1] = item_col[orig_idx + k] + else: + out_col[i * N + k - 1] = 0 + + +def apply_find_items_viewed(df, item_mappings): + + # need to sort descending to ensure that the + # next N rows are the previous N clicks + df = df.sort_values( + by=["wcs_user_sk", "tstamp", "wcs_sales_sk", "wcs_item_sk"], + ascending=[False, False, False, False], + ) + df.reset_index(drop=True, inplace=True) + df["relevant_flag"] = (df.wcs_sales_sk != 0) & ( + df.wcs_item_sk == q03_purchased_item_IN + ) + df["relevant_idx_pos"] = df.index.to_series() + df.reset_index(drop=True, inplace=True) + # only allocate output for the relevant rows + sample = df.loc[df.relevant_flag == True] + sample.reset_index(drop=True, inplace=True) + + N = q03_views_before_purchase + size = len(sample) + + # we know this can be int32, since it's going to contain item_sks + out_arr = cuda.device_array(size * N, dtype=df["wcs_item_sk"].dtype) + + load_q03() + find_items_viewed_before_purchase_kernel.forall(size)( + sample["relevant_idx_pos"], + df["wcs_user_sk"], + df["tstamp"], + df["wcs_item_sk"], + out_arr, + N, + ) + + result = cudf.DataFrame({"prior_item_viewed": out_arr}) + + del out_arr + del df + del sample + + filtered = result.merge( + item_mappings, + how="inner", + left_on=["prior_item_viewed"], + right_on=["i_item_sk"], + ) + return filtered + + +def load_q03(): + import importlib, types + + fn = os.path.join(os.getcwd(), "gpu_bdb_query_03_dask_sql.py") + if not os.path.isfile(fn): + fn = os.path.join(os.getcwd(), "queries/q03/gpu_bdb_query_03_dask_sql.py") + + loader = importlib.machinery.SourceFileLoader("03", fn) + mod = types.ModuleType(loader.name) + loader.exec_module(mod) + sys.modules[loader.name] = mod + return mod.main + diff --git a/gpu_bdb/bdb_tools/q04_utils.py b/gpu_bdb/bdb_tools/q04_utils.py new file mode 100644 index 00000000..70d75f2e --- /dev/null +++ b/gpu_bdb/bdb_tools/q04_utils.py @@ -0,0 +1,67 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import cudf + +from bdb_tools.sessionization import get_sessions + +def abandonedShoppingCarts(df, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): + + # Select groups where last dynamic row comes after last order row + filtered_df = df[ + (df["wp_type_codes"] == ORDER_CAT_CODE) + | (df["wp_type_codes"] == DYNAMIC_CAT_CODE) + ] + # Create a new column that is the concatenation of timestamp and wp_type_codes + # (eg:123456:3, 234567:5) + filtered_df["wp_type_codes"] = ( + filtered_df["tstamp_inSec"] + .astype("str") + .str.cat(filtered_df["wp_type_codes"].astype("str"), sep=":") + ) + # This gives the last occurrence (by timestamp) within the "order", "dynamic" wp_types + filtered_df = filtered_df.groupby( + ["wcs_user_sk", "session_id"], as_index=False, sort=False + ).agg({"wp_type_codes": "max"}) + # If the max contains dynamic, keep the row else discard. + last_dynamic_df = filtered_df[ + filtered_df["wp_type_codes"].str.contains( + ":" + str(DYNAMIC_CAT_CODE), regex=False + ) + ] + del filtered_df + + # Find counts for each group + grouped_count_df = df.groupby( + ["wcs_user_sk", "session_id"], as_index=False, sort=False + ).agg({"tstamp_inSec": "count"}) + # Merge counts with the "dynamic" shopping cart groups + result = last_dynamic_df.merge( + grouped_count_df, on=["wcs_user_sk", "session_id"], how="inner" + ) + del (last_dynamic_df, grouped_count_df) + return cudf.DataFrame( + {"pagecount": result.tstamp_inSec.sum(), "count": len(result)} + ) + + +def reduction_function(df, keep_cols, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): + df = get_sessions(df, keep_cols=keep_cols) + df = abandonedShoppingCarts( + df, DYNAMIC_CAT_CODE=DYNAMIC_CAT_CODE, ORDER_CAT_CODE=ORDER_CAT_CODE + ) + return df + diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index dc59ebee..4c6a3a8b 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -24,10 +24,6 @@ def load_query(qnum, fn): dask_qnums = [str(i).zfill(2) for i in range(1, 31)] sql_qnums = [str(i).zfill(2) for i in range(1, 31)] -for query in exclude: - sql_qnums = [q for q in sql_qnums if q != query] -print(sql_qnums) - if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster, import_query_libs @@ -43,7 +39,6 @@ def load_query(qnum, fn): for qnum in dask_qnums } - include_sql = True if include_sql: sql_queries = { qnum: load_query(qnum, f"queries/q{qnum}/gpu_bdb_query_{qnum}_dask_sql.py") @@ -84,19 +79,19 @@ def load_query(qnum, fn): # Run Pure Dask Queries if len(dask_qnums) > 0: print("Pure Dask Queries") - for qnum, q_func in dask_queries.items(): - print(qnum) - - qpath = f"{base_path}/queries/q{qnum}/" - os.chdir(qpath) - if os.path.exists("current_query_num.txt"): - os.remove("current_query_num.txt") - with open("current_query_num.txt", "w") as fp: - fp.write(qnum) - - for r in range(N_REPEATS): - run_query(config=config, client=client, query_func=q_func) - client.run(gc.collect) - client.run_on_scheduler(gc.collect) - gc.collect() - time.sleep(3) + for r in range(N_REPEATS): + for qnum, q_func in dask_queries.items(): + print(f"{r}: {qnum}") + + qpath = f"{base_path}/queries/q{qnum}/" + os.chdir(qpath) + if os.path.exists("current_query_num.txt"): + os.remove("current_query_num.txt") + with open("current_query_num.txt", "w") as fp: + fp.write(qnum) + + run_query(config=config, client=client, query_func=q_func) + client.run(gc.collect) + client.run_on_scheduler(gc.collect) + gc.collect() + time.sleep(3) diff --git a/gpu_bdb/benchmark_runner/benchmark_config.yaml b/gpu_bdb/benchmark_runner/benchmark_config.yaml index 54d6bfb8..db1fc4d6 100755 --- a/gpu_bdb/benchmark_runner/benchmark_config.yaml +++ b/gpu_bdb/benchmark_runner/benchmark_config.yaml @@ -6,7 +6,7 @@ file_format: parquet output_filetype: parquet split_row_groups: False repartition_small_table: True -benchmark_runner_include_bsql: +benchmark_runner_include_sql: scheduler_file_path: dask_profile: False diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index 9c9c0d2b..f025df86 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -95,5 +95,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index e2e9cb90..ecee340a 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -101,5 +101,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03.py b/gpu_bdb/queries/q03/gpu_bdb_query_03.py index 68a04af4..073d3374 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03.py @@ -14,10 +14,8 @@ # limitations under the License. # -import sys import os - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -25,6 +23,10 @@ ) from bdb_tools.readers import build_reader +from bdb_tools.q03_utils import ( + apply_find_items_viewed +) + from distributed import wait import numpy as np @@ -120,93 +122,6 @@ def read_tables(config): return item_df -@cuda.jit -def find_items_viewed_before_purchase_kernel( - relevant_idx_col, user_col, timestamp_col, item_col, out_col, N -): - """ - Find the past N items viewed after a relevant purchase was made, - as defined by the configuration of this query. - """ - i = cuda.grid(1) - relevant_item = q03_purchased_item_IN - - if i < (relevant_idx_col.size): # boundary guard - # every relevant row gets N rows in the output, so we need to map the indexes - # back into their position in the original array - orig_idx = relevant_idx_col[i] - current_user = user_col[orig_idx] - - # look at the previous N clicks (assume sorted descending) - rows_to_check = N - remaining_rows = user_col.size - orig_idx - - if remaining_rows <= rows_to_check: - rows_to_check = remaining_rows - 1 - - for k in range(1, rows_to_check + 1): - if current_user != user_col[orig_idx + k]: - out_col[i * N + k - 1] = 0 - - # only checking relevant purchases via the relevant_idx_col - elif (timestamp_col[orig_idx + k] <= timestamp_col[orig_idx]) & ( - timestamp_col[orig_idx + k] - >= (timestamp_col[orig_idx] - q03_days_in_sec_before_purchase) - ): - out_col[i * N + k - 1] = item_col[orig_idx + k] - else: - out_col[i * N + k - 1] = 0 - - -def apply_find_items_viewed(df, item_mappings): - import cudf - - # need to sort descending to ensure that the - # next N rows are the previous N clicks - df = df.sort_values( - by=["wcs_user_sk", "tstamp", "wcs_sales_sk", "wcs_item_sk"], - ascending=[False, False, False, False], - ) - df.reset_index(drop=True, inplace=True) - df["relevant_flag"] = (df.wcs_sales_sk != 0) & ( - df.wcs_item_sk == q03_purchased_item_IN - ) - df["relevant_idx_pos"] = df.index.to_series() - df.reset_index(drop=True, inplace=True) - # only allocate output for the relevant rows - sample = df.loc[df.relevant_flag == True] - sample.reset_index(drop=True, inplace=True) - - N = q03_views_before_purchase - size = len(sample) - - # we know this can be int32, since it's going to contain item_sks - out_arr = cuda.device_array(size * N, dtype=df["wcs_item_sk"].dtype) - - find_items_viewed_before_purchase_kernel.forall(size)( - sample["relevant_idx_pos"], - df["wcs_user_sk"], - df["tstamp"], - df["wcs_item_sk"], - out_arr, - N, - ) - - result = cudf.DataFrame({"prior_item_viewed": out_arr}) - - del out_arr - del df - del sample - - filtered = result.merge( - item_mappings, - how="inner", - left_on=["prior_item_viewed"], - right_on=["i_item_sk"], - ) - return filtered - - def main(client, config): import dask_cudf import cudf diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index e0ed7669..e3dc3a86 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -14,11 +14,7 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster -from numba import cuda from bdb_tools.utils import ( benchmark, @@ -28,6 +24,10 @@ from bdb_tools.readers import build_reader +from bdb_tools.q03_utils import ( + apply_find_items_viewed +) + from dask.distributed import wait @@ -40,106 +40,6 @@ q03_limit = 100 -@cuda.jit -def find_items_viewed_before_purchase_kernel( - relevant_idx_col, user_col, timestamp_col, item_col, out_col, N -): - """ - Find the past N items viewed after a relevant purchase was made, - as defined by the configuration of this query. - """ - i = cuda.grid(1) - relevant_item = q03_purchased_item_IN - - if i < (relevant_idx_col.size): # boundary guard - # every relevant row gets N rows in the output, so we need to map the indexes - # back into their position in the original array - orig_idx = relevant_idx_col[i] - current_user = user_col[orig_idx] - - # look at the previous N clicks (assume sorted descending) - rows_to_check = N - remaining_rows = user_col.size - orig_idx - - if remaining_rows <= rows_to_check: - rows_to_check = remaining_rows - 1 - - for k in range(1, rows_to_check + 1): - if current_user != user_col[orig_idx + k]: - out_col[i * N + k - 1] = 0 - - # only checking relevant purchases via the relevant_idx_col - elif (timestamp_col[orig_idx + k] <= timestamp_col[orig_idx]) & ( - timestamp_col[orig_idx + k] - >= (timestamp_col[orig_idx] - q03_days_in_sec_before_purchase) - ): - out_col[i * N + k - 1] = item_col[orig_idx + k] - else: - out_col[i * N + k - 1] = 0 - -def load_q03(): - import importlib, types - - fn = os.path.join(os.getcwd(), "gpu_bdb_query_03_dask_sql.py") - if not os.path.isfile(fn): - fn = os.path.join(os.getcwd(), "queries/q03/gpu_bdb_query_03_dask_sql.py") - - loader = importlib.machinery.SourceFileLoader("03", fn) - mod = types.ModuleType(loader.name) - loader.exec_module(mod) - sys.modules[loader.name] = mod - return mod.main - -def apply_find_items_viewed(df, item_mappings): - import cudf - - # need to sort descending to ensure that the - # next N rows are the previous N clicks - df = df.sort_values( - by=["wcs_user_sk", "tstamp", "wcs_sales_sk", "wcs_item_sk"], - ascending=[False, False, False, False], - ) - df.reset_index(drop=True, inplace=True) - df["relevant_flag"] = (df.wcs_sales_sk != 0) & ( - df.wcs_item_sk == q03_purchased_item_IN - ) - df["relevant_idx_pos"] = df.index.to_series() - df.reset_index(drop=True, inplace=True) - # only allocate output for the relevant rows - sample = df.loc[df.relevant_flag == True] - sample.reset_index(drop=True, inplace=True) - - N = q03_views_before_purchase - size = len(sample) - - # we know this can be int32, since it's going to contain item_sks - out_arr = cuda.device_array(size * N, dtype=df["wcs_item_sk"].dtype) - - load_q03() - find_items_viewed_before_purchase_kernel.forall(size)( - sample["relevant_idx_pos"], - df["wcs_user_sk"], - df["tstamp"], - df["wcs_item_sk"], - out_arr, - N, - ) - - result = cudf.DataFrame({"prior_item_viewed": out_arr}) - - del out_arr - del df - del sample - - filtered = result.merge( - item_mappings, - how="inner", - left_on=["prior_item_viewed"], - right_on=["i_item_sk"], - ) - return filtered - - def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], @@ -227,5 +127,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04.py b/gpu_bdb/queries/q04/gpu_bdb_query_04.py index 9c4cb5a5..9300ae70 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04.py @@ -16,15 +16,20 @@ import sys - from bdb_tools.utils import ( benchmark, gpubdb_argparser, run_query, ) + from bdb_tools.readers import build_reader + from bdb_tools.sessionization import get_sessions +from bdb_tools.q04_utils import ( + abandonedShoppingCarts, + reduction_function +) # parameters q04_session_timeout_inSec = 3600 @@ -52,60 +57,6 @@ def read_tables(config): return wp_df, web_clicksteams_df -def abandonedShoppingCarts(df, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): - import cudf - - # TODO: test without reset index - df.reset_index(drop=True, inplace=True) - - # Select groups where last dynamic row comes after last order row - filtered_df = df[ - (df["wp_type_codes"] == ORDER_CAT_CODE) - | (df["wp_type_codes"] == DYNAMIC_CAT_CODE) - ] - # TODO: test without reset index - filtered_df.reset_index(drop=True, inplace=True) - # Create a new column that is the concatenation of timestamp and wp_type_codes - # (eg:123456:3, 234567:5) - filtered_df["wp_type_codes"] = ( - filtered_df["tstamp_inSec"] - .astype("str") - .str.cat(filtered_df["wp_type_codes"].astype("str"), sep=":") - ) - # This gives the last occurrence (by timestamp) within the "order", "dynamic" wp_types - filtered_df = filtered_df.groupby( - ["wcs_user_sk", "session_id"], as_index=False, sort=False - ).agg({"wp_type_codes": "max"}) - # If the max contains dynamic, keep the row else discard. - last_dynamic_df = filtered_df[ - filtered_df["wp_type_codes"].str.contains( - ":" + str(DYNAMIC_CAT_CODE), regex=False - ) - ] - del filtered_df - - # Find counts for each group - grouped_count_df = df.groupby( - ["wcs_user_sk", "session_id"], as_index=False, sort=False - ).agg({"tstamp_inSec": "count"}) - # Merge counts with the "dynamic" shopping cart groups - result = last_dynamic_df.merge( - grouped_count_df, on=["wcs_user_sk", "session_id"], how="inner" - ) - del (last_dynamic_df, grouped_count_df) - return cudf.DataFrame( - {"pagecount": result.tstamp_inSec.sum(), "count": len(result)} - ) - - -def reduction_function(df, keep_cols, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): - df = get_sessions(df, keep_cols=keep_cols) - df = abandonedShoppingCarts( - df, DYNAMIC_CAT_CODE=DYNAMIC_CAT_CODE, ORDER_CAT_CODE=ORDER_CAT_CODE - ) - return df - - def main(client, config): import cudf diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index f2aabfae..785a5c2e 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -18,7 +18,6 @@ import os from bdb_tools.cluster_startup import attach_to_cluster -from bdb_tools.sessionization import get_sessions from bdb_tools.utils import ( benchmark, @@ -28,60 +27,12 @@ from bdb_tools.readers import build_reader -from dask.distributed import wait - - -def abandonedShoppingCarts(df, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): - import cudf - # work around for https://github.com/rapidsai/cudf/issues/5470 - df.reset_index(drop=True, inplace=True) - - # Select groups where last dynamic row comes after last order row - filtered_df = df[ - (df["wp_type_codes"] == ORDER_CAT_CODE) - | (df["wp_type_codes"] == DYNAMIC_CAT_CODE) - ] - # work around for https://github.com/rapidsai/cudf/issues/5470 - filtered_df.reset_index(drop=True, inplace=True) - # Create a new column that is the concatenation of timestamp and wp_type_codes - # (eg:123456:3, 234567:5) - filtered_df["wp_type_codes"] = ( - filtered_df["tstamp_inSec"] - .astype("str") - .str.cat(filtered_df["wp_type_codes"].astype("str"), sep=":") - ) - # This gives the last occurrence (by timestamp) within the "order", "dynamic" wp_types - filtered_df = filtered_df.groupby( - ["wcs_user_sk", "session_id"], as_index=False, sort=False - ).agg({"wp_type_codes": "max"}) - # If the max contains dynamic, keep the row else discard. - last_dynamic_df = filtered_df[ - filtered_df["wp_type_codes"].str.contains( - ":" + str(DYNAMIC_CAT_CODE), regex=False - ) - ] - del filtered_df - - # Find counts for each group - grouped_count_df = df.groupby( - ["wcs_user_sk", "session_id"], as_index=False, sort=False - ).agg({"tstamp_inSec": "count"}) - # Merge counts with the "dynamic" shopping cart groups - result = last_dynamic_df.merge( - grouped_count_df, on=["wcs_user_sk", "session_id"], how="inner" - ) - del (last_dynamic_df, grouped_count_df) - return cudf.DataFrame( - {"pagecount": result.tstamp_inSec.sum(), "count": len(result)} - ) - +from bdb_tools.q04_utils import ( + abandonedShoppingCarts, + reduction_function +) -def reduction_function(df, keep_cols, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): - df = get_sessions(df, keep_cols=keep_cols) - df = abandonedShoppingCarts( - df, DYNAMIC_CAT_CODE=DYNAMIC_CAT_CODE, ORDER_CAT_CODE=ORDER_CAT_CODE - ) - return df +from dask.distributed import wait def read_tables(data_dir, c, config): @@ -120,9 +71,7 @@ def main(data_dir, client, c, config): wp["wp_type"] = wp["wp_type"].map_partitions( lambda ser: ser.astype("category")) - cpu_categories = wp["wp_type"].compute().cat.categories - if hasattr(cpu_categories, "to_pandas"): - cpu_categories = cpu_categories.to_pandas() + cpu_categories = wp["wp_type"].compute().cat.categories.to_pandas() DYNAMIC_CAT_CODE = cpu_categories.get_loc("dynamic") ORDER_CAT_CODE = cpu_categories.get_loc("order") @@ -171,5 +120,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index 7ccf34e3..19b540f7 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -176,5 +176,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index e81c8ff3..2ab18d2c 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -150,6 +150,6 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index 47ccab2f..58fcdbf1 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -105,5 +105,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 9afa57b9..33dcafc6 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -276,5 +276,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index 8f7355a1..8c5ef131 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -168,5 +168,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index e00022a5..5f8cff83 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -161,5 +161,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index 2efd1514..7b49d7f0 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -97,5 +97,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index 3db7d4ce..c3e93b1c 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -89,5 +89,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index 68d14d44..2ac68b3e 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -127,5 +127,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index d34a367d..b5cd4873 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -83,5 +83,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index 19d855b0..3c06393d 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -95,5 +95,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 9a78c428..2964bdd6 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -125,5 +125,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index ccbe050b..9c20982d 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -126,5 +126,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index 8176e9c7..3d15cf2e 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -350,5 +350,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index 0a767053..b2c46e10 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -202,5 +202,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index 68a87fe1..db8a54eb 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -159,5 +159,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index bfe53354..626f376c 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -159,5 +159,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index 43b4a9bf..8f0d6815 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -116,5 +116,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index b97114d4..020c00cd 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -120,5 +120,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index 07feaf87..3a8dbfe1 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -111,5 +111,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index 74443bb0..3cc228f6 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -213,5 +213,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 09adcff0..2f3aae2a 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -121,5 +121,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index eddc82c1..9c1b8046 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -142,6 +142,6 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 787d00d8..77966dd4 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -362,5 +362,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 85a0d629..d11e109d 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -99,5 +99,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index aa3db4db..7468dec8 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -73,8 +73,6 @@ def main(data_dir, client, c, config): item_df = item_df.persist() wait(item_df) c.create_table("item_df", item_df, persist=False) - # print(len(item_df)) - # print(len(item_df.columns)) query_2 = """ SELECT wcs_user_sk, @@ -122,5 +120,5 @@ def main(data_dir, client, c, config): if __name__ == "__main__": config = gpubdb_argparser() - client, c = attach_to_cluster(config) + client, c = attach_to_cluster(config, create_sql_context=True) run_query(config=config, client=client, query_func=main, sql_context=c) From 6637e11d6d4010967b91198839b65e1424537a74 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Thu, 6 Jan 2022 20:16:11 +0000 Subject: [PATCH 32/51] Remove load_q03 --- gpu_bdb/bdb_tools/q03_utils.py | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/gpu_bdb/bdb_tools/q03_utils.py b/gpu_bdb/bdb_tools/q03_utils.py index 1d02e6d6..290b9d24 100644 --- a/gpu_bdb/bdb_tools/q03_utils.py +++ b/gpu_bdb/bdb_tools/q03_utils.py @@ -88,7 +88,6 @@ def apply_find_items_viewed(df, item_mappings): # we know this can be int32, since it's going to contain item_sks out_arr = cuda.device_array(size * N, dtype=df["wcs_item_sk"].dtype) - load_q03() find_items_viewed_before_purchase_kernel.forall(size)( sample["relevant_idx_pos"], df["wcs_user_sk"], @@ -112,17 +111,3 @@ def apply_find_items_viewed(df, item_mappings): ) return filtered - -def load_q03(): - import importlib, types - - fn = os.path.join(os.getcwd(), "gpu_bdb_query_03_dask_sql.py") - if not os.path.isfile(fn): - fn = os.path.join(os.getcwd(), "queries/q03/gpu_bdb_query_03_dask_sql.py") - - loader = importlib.machinery.SourceFileLoader("03", fn) - mod = types.ModuleType(loader.name) - loader.exec_module(mod) - sys.modules[loader.name] = mod - return mod.main - From 0720a664994afbfb6a1c5654cb92d51fe9bdbce1 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Tue, 11 Jan 2022 07:02:29 +0000 Subject: [PATCH 33/51] Share code between sql and Dask queries --- gpu_bdb/bdb_tools/q05_utils.py | 72 +++++ gpu_bdb/bdb_tools/q08_utils.py | 123 ++++++++ gpu_bdb/bdb_tools/q18_utils.py | 89 ++++++ gpu_bdb/bdb_tools/q20_utils.py | 49 +++ gpu_bdb/bdb_tools/q27_utils.py | 31 ++ gpu_bdb/bdb_tools/q28_utils.py | 287 ++++++++++++++++++ gpu_bdb/queries/q05/gpu_bdb_query_05.py | 55 +--- .../queries/q05/gpu_bdb_query_05_dask_sql.py | 59 +--- gpu_bdb/queries/q08/gpu_bdb_query_08.py | 113 +------ .../queries/q08/gpu_bdb_query_08_dask_sql.py | 119 +------- gpu_bdb/queries/q18/gpu_bdb_query_18.py | 80 +---- .../queries/q18/gpu_bdb_query_18_dask_sql.py | 79 +---- gpu_bdb/queries/q20/gpu_bdb_query_20.py | 38 +-- .../queries/q20/gpu_bdb_query_20_dask_sql.py | 32 +- gpu_bdb/queries/q27/gpu_bdb_query_27.py | 36 +-- .../queries/q27/gpu_bdb_query_27_dask_sql.py | 20 +- gpu_bdb/queries/q28/gpu_bdb_query_28.py | 267 +--------------- .../queries/q28/gpu_bdb_query_28_dask_sql.py | 273 +---------------- 18 files changed, 694 insertions(+), 1128 deletions(-) create mode 100644 gpu_bdb/bdb_tools/q05_utils.py create mode 100644 gpu_bdb/bdb_tools/q08_utils.py create mode 100644 gpu_bdb/bdb_tools/q18_utils.py create mode 100644 gpu_bdb/bdb_tools/q20_utils.py create mode 100644 gpu_bdb/bdb_tools/q27_utils.py create mode 100644 gpu_bdb/bdb_tools/q28_utils.py diff --git a/gpu_bdb/bdb_tools/q05_utils.py b/gpu_bdb/bdb_tools/q05_utils.py new file mode 100644 index 00000000..c6ded04b --- /dev/null +++ b/gpu_bdb/bdb_tools/q05_utils.py @@ -0,0 +1,72 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import cupy as cp + +import cuml +from cuml.metrics import confusion_matrix + +from bdb_tools.cupy_metrics import cupy_precision_score + +from sklearn.metrics import roc_auc_score + +# Logistic Regression params +# solver = "LBFGS" Used by passing `penalty=None` or "l2" +# step_size = 1 Not used +# numCorrections = 10 Not used +iterations = 100 +C = 10_000 # reg_lambda = 0 hence C for model is a large value +convergence_tol = 1e-9 + + +def build_and_predict_model(ml_input_df): + """ + Create a standardized feature matrix X and target array y. + Returns the model and accuracy statistics + """ + + feature_names = ["college_education", "male"] + [ + "clicks_in_%d" % i for i in range(1, 8) + ] + X = ml_input_df[feature_names] + # Standardize input matrix + X = (X - X.mean()) / X.std() + y = ml_input_df["clicks_in_category"] + + model = cuml.LogisticRegression( + tol=convergence_tol, + penalty="none", + solver="qn", + fit_intercept=True, + max_iter=iterations, + C=C, + ) + model.fit(X, y) + # + # Predict and evaluate accuracy + # (Should be 1.0) at SF-1 + # + results_dict = {} + y_pred = model.predict(X) + + results_dict["auc"] = roc_auc_score(y.to_array(), y_pred.to_array()) + results_dict["precision"] = cupy_precision_score(cp.asarray(y), cp.asarray(y_pred)) + results_dict["confusion_matrix"] = confusion_matrix( + cp.asarray(y, dtype="int32"), cp.asarray(y_pred, dtype="int32") + ) + results_dict["output_type"] = "supervised" + return results_dict + diff --git a/gpu_bdb/bdb_tools/q08_utils.py b/gpu_bdb/bdb_tools/q08_utils.py new file mode 100644 index 00000000..4766beb6 --- /dev/null +++ b/gpu_bdb/bdb_tools/q08_utils.py @@ -0,0 +1,123 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import cudf + +import cupy as cp +import numpy as np + +q08_SECONDS_BEFORE_PURCHASE = 259200 +NA_FLAG = 0 + +def get_session_id_from_session_boundary(session_change_df, last_session_len): + """ + This function returns session starts given a session change df + """ + + user_session_ids = session_change_df.tstamp_inSec + + ### up shift the session length df + session_len = session_change_df["t_index"].diff().reset_index(drop=True) + session_len = session_len.shift(-1) + + try: + session_len.iloc[-1] = last_session_len + except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 + session_len = cudf.Series([]) + + session_id_final_series = ( + cudf.Series(user_session_ids).repeat(session_len).reset_index(drop=True) + ) + return session_id_final_series + + +def get_session_id(df): + """ + This function creates a session id column for each click + The session id grows in incremeant for each user's susbequent session + Session boundry is defined by the time_out + """ + + df["user_change_flag"] = df["wcs_user_sk"].diff(periods=1) != 0 + df["user_change_flag"] = df["user_change_flag"].fillna(True) + df["session_change_flag"] = df["review_flag"] | df["user_change_flag"] + + df = df.reset_index(drop=True) + df["t_index"] = cp.arange(start=0, stop=len(df), dtype=np.int32) + + session_change_df = df[df["session_change_flag"]].reset_index(drop=True) + try: + last_session_len = len(df) - session_change_df["t_index"].iloc[-1] + except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 + last_session_len = 0 + + session_ids = get_session_id_from_session_boundary( + session_change_df, last_session_len + ) + + assert len(session_ids) == len(df) + return session_ids + + +def get_sessions(df): + df = df.sort_values( + by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] + ).reset_index(drop=True) + df["session_id"] = get_session_id(df) + return df + + +def get_unique_sales_keys_from_sessions(sessionized, review_cat_code): + sessionized["relevant"] = ( + (sessionized.tstamp_inSec - sessionized.session_id) + <= q08_SECONDS_BEFORE_PURCHASE + ) & (sessionized.wcs_sales_sk != NA_FLAG) + unique_sales_sk = ( + sessionized.query(f"wcs_sales_sk != {NA_FLAG}") + .query("relevant == True") + .query(f"wp_type_codes != {review_cat_code}") + .wcs_sales_sk.unique() + ) + + return unique_sales_sk + + +def prep_for_sessionization(df, review_cat_code): + df = df.fillna(NA_FLAG) + df = df.sort_values( + by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] + ).reset_index(drop=True) + + review_df = df.loc[df["wp_type_codes"] == review_cat_code] + # per user, the index of the first review + # need this to decide if a review was "recent enough" + every_users_first_review = ( + review_df[["wcs_user_sk", "tstamp_inSec"]] + .drop_duplicates() + .reset_index() + .groupby("wcs_user_sk")["index"] + .min() + .reset_index() + ) + every_users_first_review.columns = ["wcs_user_sk", "first_review_index"] + + # then reset the index to keep the old index before parallel join + df_merged = df.reset_index().merge( + every_users_first_review, how="left", on="wcs_user_sk" + ) + df_filtered = df_merged.query("index >= first_review_index") + return df_filtered + diff --git a/gpu_bdb/bdb_tools/q18_utils.py b/gpu_bdb/bdb_tools/q18_utils.py new file mode 100644 index 00000000..998adcfb --- /dev/null +++ b/gpu_bdb/bdb_tools/q18_utils.py @@ -0,0 +1,89 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import cupy as cp +import cudf +from cudf._lib.strings import find_multiple + +def create_found_reshaped_with_global_pos(found, targets): + """Given the dataframe created by mapping find_targets_in_reviews, + create a new dataframe in which the nonzero values in each row are exploded + to get their own row. Each row will contain the word, its mapping in the column order, + and the pr_review_sk for the review from which it came. + Having these as two separate functions makes managing dask metadata easier. + """ + + target_df = cudf.DataFrame({"word": targets}).reset_index(drop=False) + target_df.columns = ["word_mapping", "word"] + + df_clean = found.drop(["pr_review_sk"], axis=1) + + row_idxs, col_idxs = df_clean.values.nonzero() + + found_reshaped = cudf.DataFrame( + {"word_mapping": col_idxs, "pr_review_sk": found["pr_review_sk"].iloc[row_idxs]} + ) + found_reshaped = found_reshaped.merge(target_df, on="word_mapping", how="inner")[ + ["word", "pr_review_sk"] + ] + return found_reshaped + + +def find_targets_in_reviews_helper(ddf, targets, str_col_name="pr_review_content"): + """returns a N x K matrix, where N is the number of rows in ddf that + contain one of the target words and K is the number of words in targets. + + If a target word is found in a review, the value in that row, column + is non-zero. + + At the end, any row with non-zero values is returned. + + """ + + lowered = ddf[str_col_name].str.lower() + + ## TODO: Do the replace/any in cupy land before going to cuDF + resdf = cudf.DataFrame( + cp.asarray( + find_multiple.find_multiple(lowered._column, targets._column) + ).reshape(-1, len(targets)) + ) + + resdf = resdf.replace([0, -1], [1, 0]) + found_mask = resdf.any(axis=1) + resdf["pr_review_sk"] = ddf["pr_review_sk"] + found = resdf.loc[found_mask] + return create_found_reshaped_with_global_pos(found, targets) + + +def find_relevant_reviews(df, targets, str_col_name="pr_review_content"): + """ + This function finds the reviews containg target stores and returns the + relevant reviews + """ + + targets = cudf.Series(targets) + targets_lower = targets.str.lower() + reviews_found = find_targets_in_reviews_helper(df, targets_lower)[ + ["word", "pr_review_sk"] + ] + + combined = reviews_found.merge( + df[["pr_review_date", "pr_review_sk"]], how="inner", on=["pr_review_sk"] + ) + + return combined + diff --git a/gpu_bdb/bdb_tools/q20_utils.py b/gpu_bdb/bdb_tools/q20_utils.py new file mode 100644 index 00000000..5871f93e --- /dev/null +++ b/gpu_bdb/bdb_tools/q20_utils.py @@ -0,0 +1,49 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import dask_cudf + +from dask import delayed + +from bdb_tools.utils import train_clustering_model + +# q20 parameters +N_CLUSTERS = 8 +CLUSTER_ITERATIONS = 20 +N_ITER = 5 + +def get_clusters(client, ml_input_df, feature_cols): + """ + Takes the dask client, kmeans_input_df and feature columns. + Returns a dictionary matching the output required for q20 + """ + ml_tasks = [ + delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) + for df in ml_input_df[feature_cols].to_delayed() + ] + + results_dict = client.compute(*ml_tasks, sync=True) + + labels = results_dict["cid_labels"] + + labels_final = dask_cudf.from_cudf(labels, npartitions=ml_input_df.npartitions) + ml_input_df["label"] = labels_final.reset_index()[0] + + output = ml_input_df[["user_sk", "label"]] + + results_dict["cid_labels"] = output + return results_dict + diff --git a/gpu_bdb/bdb_tools/q27_utils.py b/gpu_bdb/bdb_tools/q27_utils.py new file mode 100644 index 00000000..499135b1 --- /dev/null +++ b/gpu_bdb/bdb_tools/q27_utils.py @@ -0,0 +1,31 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import spacy + +def ner_parser(df, col_string, batch_size=256): + spacy.require_gpu() + nlp = spacy.load("en_core_web_sm") + docs = nlp.pipe(df[col_string], disable=["tagger", "parser"], batch_size=batch_size) + out = [] + for doc in docs: + l = [ent.text for ent in doc.ents if ent.label_ == "ORG"] + val = ", " + l = val.join(l) + out.append(l) + df["company_name_list"] = out + return df + diff --git a/gpu_bdb/bdb_tools/q28_utils.py b/gpu_bdb/bdb_tools/q28_utils.py new file mode 100644 index 00000000..7d3d15ac --- /dev/null +++ b/gpu_bdb/bdb_tools/q28_utils.py @@ -0,0 +1,287 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import time + +import numpy as np +import cupy as cp +import cupy + +import cudf + +import dask + +from cuml.feature_extraction.text import HashingVectorizer +from cuml.dask.naive_bayes import MultinomialNB as DistMNB +from cuml.dask.common import to_dask_cudf +from cuml.dask.common.input_utils import DistributedDataHandler + +from distributed import wait + +from uuid import uuid1 + + +N_FEATURES = 2 ** 23 # Spark is doing 2^20 +ngram_range = (1, 2) +preprocessor = lambda s:s.str.lower() +norm = None +alternate_sign = False + + +def gpu_hashing_vectorizer(x): + vec = HashingVectorizer(n_features=N_FEATURES, + alternate_sign=alternate_sign, + ngram_range=ngram_range, + norm=norm, + preprocessor=preprocessor + ) + return vec.fit_transform(x) + + +def map_labels(ser): + output_ser = cudf.Series(cudf.core.column.full(size=len(ser), fill_value=2, dtype=np.int32)) + zero_flag = (ser==1) | (ser==2) + output_ser.loc[zero_flag]=0 + + three_flag = (ser==3) + output_ser.loc[three_flag]=1 + + return output_ser + +def build_features(t): + X = t["pr_review_content"] + X = X.map_partitions( + gpu_hashing_vectorizer, + meta=dask.array.from_array( + cupy.sparse.csr_matrix(cupy.zeros(1, dtype=cp.float32)) + ), + ) + + X = X.astype(np.float32).persist() + X.compute_chunk_sizes() + + return X + + +def build_labels(reviews_df): + y = reviews_df["pr_review_rating"].map_partitions(map_labels) + y = y.map_partitions(lambda x: cupy.asarray(x, cupy.int32)).persist() + y.compute_chunk_sizes() + + return y + +def categoricalize(num_sr): + return num_sr.astype("str").str.replace(["0", "1", "2"], ["NEG", "NEUT", "POS"]) + + +def sum_tp_fp(y_y_pred, nclasses): + + y, y_pred = y_y_pred + res = cp.zeros((nclasses, 2), order="F") + + for i in range(nclasses): + pos_pred_ix = cp.where(y_pred == i)[0] + + # short circuit + if len(pos_pred_ix) == 0: + res[i] = 0 + break + + tp_sum = (y_pred[pos_pred_ix] == y[pos_pred_ix]).sum() + fp_sum = (y_pred[pos_pred_ix] != y[pos_pred_ix]).sum() + res[i][0] = tp_sum + res[i][1] = fp_sum + return res + +def precision_score(client, y, y_pred, average="binary"): + + nclasses = len(cp.unique(y.map_blocks(lambda x: cp.unique(x)).compute())) + + if average == "binary" and nclasses > 2: + raise ValueError + + if nclasses < 2: + raise ValueError("Single class precision is not yet supported") + + ddh = DistributedDataHandler.create([y, y_pred]) + + precision_scores = client.compute( + [ + client.submit(sum_tp_fp, part, nclasses, workers=[worker]) + for worker, part in ddh.gpu_futures + ], + sync=True, + ) + + res = cp.zeros((nclasses, 2), order="F") + + for i in precision_scores: + res += i + + if average == "binary" or average == "macro": + + prec = cp.zeros(nclasses) + for i in range(nclasses): + tp_sum, fp_sum = res[i] + prec[i] = (tp_sum / (tp_sum + fp_sum)).item() + + if average == "binary": + return prec[nclasses - 1].item() + else: + return prec.mean().item() + else: + global_tp = cp.sum(res[:, 0]) + global_fp = cp.sum(res[:, 1]) + + return global_tp / (global_tp + global_fp).item() + + +def local_cm(y_y_pred, unique_labels, sample_weight): + + y_true, y_pred = y_y_pred + labels = unique_labels + + n_labels = labels.size + + # Assume labels are monotonically increasing for now. + + # intersect y_pred, y_true with labels, eliminate items not in labels + ind = cp.logical_and(y_pred < n_labels, y_true < n_labels) + y_pred = y_pred[ind] + y_true = y_true[ind] + + if sample_weight is None: + sample_weight = cp.ones(y_true.shape[0], dtype=np.int64) + else: + sample_weight = cp.asarray(sample_weight) + + sample_weight = sample_weight[ind] + + cm = cp.sparse.coo_matrix( + (sample_weight, (y_true, y_pred)), shape=(n_labels, n_labels), dtype=cp.float32, + ).toarray() + + return cp.nan_to_num(cm) + + +def confusion_matrix(client, y_true, y_pred, normalize=None, sample_weight=None): + + unique_classes = cp.unique(y_true.map_blocks(lambda x: cp.unique(x)).compute()) + nclasses = len(unique_classes) + + ddh = DistributedDataHandler.create([y_true, y_pred]) + + cms = client.compute( + [ + client.submit( + local_cm, part, unique_classes, sample_weight, workers=[worker] + ) + for worker, part in ddh.gpu_futures + ], + sync=True, + ) + + cm = cp.zeros((nclasses, nclasses)) + for i in cms: + cm += i + + with np.errstate(all="ignore"): + if normalize == "true": + cm = cm / cm.sum(axis=1, keepdims=True) + elif normalize == "pred": + cm = cm / cm.sum(axis=0, keepdims=True) + elif normalize == "all": + cm = cm / cm.sum() + cm = cp.nan_to_num(cm) + + return cm + + +def accuracy_score(client, y, y_hat): + + ddh = DistributedDataHandler.create([y_hat, y]) + + def _count_accurate_predictions(y_hat_y): + y_hat, y = y_hat_y + y_hat = cp.asarray(y_hat, dtype=y_hat.dtype) + y = cp.asarray(y, dtype=y.dtype) + return y.shape[0] - cp.count_nonzero(y - y_hat) + + key = uuid1() + + futures = client.compute( + [ + client.submit( + _count_accurate_predictions, + worker_future[1], + workers=[worker_future[0]], + key="%s-%s" % (key, idx), + ) + for idx, worker_future in enumerate(ddh.gpu_futures) + ], + sync=True, + ) + + return sum(futures) / y.shape[0] + + +def post_etl_processing(client, train_data, test_data): + + # Feature engineering + X_train = build_features(train_data) + X_test = build_features(test_data) + + y_train = build_labels(train_data) + y_test = build_labels(test_data) + + # Perform ML + model = DistMNB(client=client, alpha=0.001) + model.fit(X_train, y_train) + + ### this regression seems to be coming from here + test_pred_st = time.time() + y_hat = model.predict(X_test).persist() + + # Compute distributed performance metrics + acc = accuracy_score(client, y_test, y_hat) + + print("Accuracy: " + str(acc)) + prec = precision_score(client, y_test, y_hat, average="macro") + + print("Precision: " + str(prec)) + cmat = confusion_matrix(client, y_test, y_hat) + + print("Confusion Matrix: " + str(cmat)) + metric_et = time.time() + + # Place results back in original Dataframe + + ddh = DistributedDataHandler.create(y_hat) + test_preds = to_dask_cudf( + [client.submit(cudf.Series, part) for w, part in ddh.gpu_futures] + ) + + test_preds = test_preds.map_partitions(categoricalize) + + test_data["prediction"] = test_preds + + final_data = test_data[["pr_review_sk", "pr_review_rating", "prediction"]].persist() + + final_data = final_data.sort_values("pr_review_sk").reset_index(drop=True) + wait(final_data) + return final_data, acc, prec, cmat + + diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05.py b/gpu_bdb/queries/q05/gpu_bdb_query_05.py index 290cf127..3d96122b 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -14,7 +14,6 @@ # limitations under the License. # -import sys import os import glob @@ -25,14 +24,13 @@ ) from bdb_tools.readers import build_reader -from bdb_tools.cupy_metrics import cupy_precision_score +from bdb_tools.q05_utils import build_and_predict_model import cupy as cp import numpy as np from dask import delayed import dask import pandas as pd -from sklearn.metrics import roc_auc_score # # Query Configuration @@ -45,14 +43,6 @@ customer_columns = ["c_customer_sk", "c_current_cdemo_sk"] customer_dem_columns = ["cd_demo_sk", "cd_gender", "cd_education_status"] -# Logistic Regression params -# solver = "LBFGS" Used by passing `penalty=None` or "l2" -# step_size = 1 Not used -# numCorrections = 10 Not used -iterations = 100 -C = 10_000 # reg_lambda = 0 hence C for model is a large value -convergence_tol = 1e-9 - def read_tables(config): table_reader = build_reader( @@ -72,47 +62,6 @@ def read_tables(config): return (item_ddf, customer_ddf, customer_dem_ddf) -def build_and_predict_model(ml_input_df): - """ - Create a standardized feature matrix X and target array y. - Returns the model and accuracy statistics - """ - import cuml - from cuml.metrics import confusion_matrix - - feature_names = ["college_education", "male"] + [ - "clicks_in_%d" % i for i in range(1, 8) - ] - X = ml_input_df[feature_names] - # Standardize input matrix - X = (X - X.mean()) / X.std() - y = ml_input_df["clicks_in_category"] - - model = cuml.LogisticRegression( - tol=convergence_tol, - penalty="none", - solver="qn", - fit_intercept=True, - max_iter=iterations, - C=C, - ) - model.fit(X, y) - # - # Predict and evaluate accuracy - # (Should be 1.0) at SF-1 - # - results_dict = {} - y_pred = model.predict(X) - - results_dict["auc"] = roc_auc_score(y.to_array(), y_pred.to_array()) - results_dict["precision"] = cupy_precision_score(cp.asarray(y), cp.asarray(y_pred)) - results_dict["confusion_matrix"] = confusion_matrix( - cp.asarray(y, dtype="int32"), cp.asarray(y_pred, dtype="int32") - ) - results_dict["output_type"] = "supervised" - return results_dict - - def get_groupby_results(file_list, item_df): """ Functionial approach for better scaling diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index 19b540f7..3eb4dc4a 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -14,13 +14,9 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -from dask_cuda import LocalCUDACluster -from dask.distributed import Client, wait +from dask.distributed import wait from dask import delayed -import os from bdb_tools.utils import ( benchmark, @@ -28,18 +24,8 @@ run_query, ) from bdb_tools.readers import build_reader -from bdb_tools.cupy_metrics import cupy_precision_score -from sklearn.metrics import roc_auc_score -import cupy as cp - +from bdb_tools.q05_utils import build_and_predict_model -# Logistic Regression params -# solver = "LBFGS" Used by passing `penalty=None` or "l2" -# step_size = 1 Not used -# numCorrections = 10 Not used -iterations = 100 -C = 10_000 # reg_lambda = 0 hence C for model is a large value -convergence_tol = 1e-9 wcs_columns = ["wcs_item_sk", "wcs_user_sk"] items_columns = ["i_item_sk", "i_category", "i_category_id"] @@ -70,47 +56,6 @@ def read_tables(data_dir, c, config): c.create_table("customer_demographics", customer_dem_ddf, persist=False) -def build_and_predict_model(ml_input_df): - """ - Create a standardized feature matrix X and target array y. - Returns the model and accuracy statistics - """ - import cuml - from cuml.metrics import confusion_matrix - - feature_names = ["college_education", "male"] + [ - "clicks_in_%d" % i for i in range(1, 8) - ] - X = ml_input_df[feature_names] - # Standardize input matrix - X = (X - X.mean()) / X.std() - y = ml_input_df["clicks_in_category"] - - model = cuml.LogisticRegression( - tol=convergence_tol, - penalty="none", - solver="qn", - fit_intercept=True, - max_iter=iterations, - C=C, - ) - model.fit(X, y) - # - # Predict and evaluate accuracy - # (Should be 1.0) at SF-1 - # - results_dict = {} - y_pred = model.predict(X) - - results_dict["auc"] = roc_auc_score(y.to_array(), y_pred.to_array()) - results_dict["precision"] = cupy_precision_score(cp.asarray(y), cp.asarray(y_pred)) - results_dict["confusion_matrix"] = confusion_matrix( - cp.asarray(y, dtype="int32"), cp.asarray(y_pred, dtype="int32") - ) - results_dict["output_type"] = "supervised" - return results_dict - - def main(data_dir, client, c, config): benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08.py b/gpu_bdb/queries/q08/gpu_bdb_query_08.py index 686ea05b..cc7b725e 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -14,7 +14,6 @@ # limitations under the License. # -import sys import os import glob @@ -26,18 +25,19 @@ ) from bdb_tools.readers import build_reader from bdb_tools.merge_util import hash_merge +from bdb_tools.q08_utils import ( + get_sessions, + get_unique_sales_keys_from_sessions, + prep_for_sessionization, +) import numpy as np from distributed import wait -import cupy as cp -import rmm from dask import delayed q08_STARTDATE = "2001-09-02" q08_ENDDATE = "2002-09-02" -q08_SECONDS_BEFORE_PURCHASE = 259200 -NA_FLAG = 0 def etl_wcs(wcs_fn, filtered_date_df, web_page_df): @@ -81,107 +81,6 @@ def etl_wcs(wcs_fn, filtered_date_df, web_page_df): return merged_df[cols_to_keep] -def get_session_id_from_session_boundary(session_change_df, last_session_len): - """ - This function returns session starts given a session change df - """ - import cudf - - user_session_ids = session_change_df.tstamp_inSec - - ### up shift the session length df - session_len = session_change_df["t_index"].diff().reset_index(drop=True) - session_len = session_len.shift(-1) - - try: - session_len.iloc[-1] = last_session_len - except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 - session_len = cudf.Series([]) - - session_id_final_series = ( - cudf.Series(user_session_ids).repeat(session_len).reset_index(drop=True) - ) - return session_id_final_series - - -def get_session_id(df): - """ - This function creates a session id column for each click - The session id grows in incremeant for each user's susbequent session - Session boundry is defined by the time_out - """ - - df["user_change_flag"] = df["wcs_user_sk"].diff(periods=1) != 0 - df["user_change_flag"] = df["user_change_flag"].fillna(True) - df["session_change_flag"] = df["review_flag"] | df["user_change_flag"] - - df = df.reset_index(drop=True) - df["t_index"] = cp.arange(start=0, stop=len(df), dtype=np.int32) - - session_change_df = df[df["session_change_flag"]].reset_index(drop=True) - try: - last_session_len = len(df) - session_change_df["t_index"].iloc[-1] - except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 - last_session_len = 0 - - session_ids = get_session_id_from_session_boundary( - session_change_df, last_session_len - ) - - assert len(session_ids) == len(df) - return session_ids - - -def get_sessions(df): - df = df.sort_values( - by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] - ).reset_index(drop=True) - df["session_id"] = get_session_id(df) - return df - - -def get_unique_sales_keys_from_sessions(sessionized, review_cat_code): - sessionized["relevant"] = ( - (sessionized.tstamp_inSec - sessionized.session_id) - <= q08_SECONDS_BEFORE_PURCHASE - ) & (sessionized.wcs_sales_sk != NA_FLAG) - unique_sales_sk = ( - sessionized.query(f"wcs_sales_sk != {NA_FLAG}") - .query("relevant == True") - .query(f"wp_type_codes != {review_cat_code}") - .wcs_sales_sk.unique() - ) - - return unique_sales_sk - - -def prep_for_sessionization(df, review_cat_code): - df = df.fillna(NA_FLAG) - df = df.sort_values( - by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] - ).reset_index(drop=True) - - review_df = df.loc[df["wp_type_codes"] == review_cat_code] - # per user, the index of the first review - # need this to decide if a review was "recent enough" - every_users_first_review = ( - review_df[["wcs_user_sk", "tstamp_inSec"]] - .drop_duplicates() - .reset_index() - .groupby("wcs_user_sk")["index"] - .min() - .reset_index() - ) - every_users_first_review.columns = ["wcs_user_sk", "first_review_index"] - - # then reset the index to keep the old index before parallel join - df_merged = df.reset_index().merge( - every_users_first_review, how="left", on="wcs_user_sk" - ) - df_filtered = df_merged.query("index >= first_review_index") - return df_filtered - - def read_tables(config): table_reader = build_reader( data_format=config["file_format"], diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 33dcafc6..634c1305 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -14,14 +14,12 @@ # limitations under the License. # -import sys import os -from bdb_tools.cluster_startup import attach_to_cluster -import cupy as cp -import numpy as np import cudf +from bdb_tools.cluster_startup import attach_to_cluster + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -30,118 +28,19 @@ from bdb_tools.readers import build_reader +from bdb_tools.q08_utils import ( + get_sessions, + get_unique_sales_keys_from_sessions, + prep_for_sessionization, +) + from dask.distributed import wait # -------- Q8 ----------- -q08_SECONDS_BEFORE_PURCHASE = 259200 q08_STARTDATE = "2001-09-02" q08_ENDDATE = "2002-09-02" -REVIEW_CAT_CODE = 6 -NA_FLAG = 0 - - -def get_session_id_from_session_boundary(session_change_df, last_session_len): - """ - This function returns session starts given a session change df - """ - import cudf - - user_session_ids = session_change_df.tstamp_inSec - - ### up shift the session length df - session_len = session_change_df["t_index"].diff().reset_index(drop=True) - session_len = session_len.shift(-1) - - try: - session_len.iloc[-1] = last_session_len - except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 - session_len = cudf.Series([]) - - session_id_final_series = ( - cudf.Series(user_session_ids).repeat(session_len).reset_index(drop=True) - ) - return session_id_final_series - - -def get_session_id(df): - """ - This function creates a session id column for each click - The session id grows in incremeant for each user's susbequent session - Session boundry is defined by the time_out - """ - - df["user_change_flag"] = df["wcs_user_sk"].diff(periods=1) != 0 - df["user_change_flag"] = df["user_change_flag"].fillna(True) - df["session_change_flag"] = df["review_flag"] | df["user_change_flag"] - - df = df.reset_index(drop=True) - df["t_index"] = cp.arange(start=0, stop=len(df), dtype=np.int32) - - session_change_df = df[df["session_change_flag"]].reset_index(drop=True) - try: - last_session_len = len(df) - session_change_df["t_index"].iloc[-1] - except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 - last_session_len = 0 - - session_ids = get_session_id_from_session_boundary( - session_change_df, last_session_len - ) - - assert len(session_ids) == len(df) - return session_ids - - -def get_sessions(df): - df = df.sort_values( - by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] - ).reset_index(drop=True) - df["session_id"] = get_session_id(df) - return df - - -def get_unique_sales_keys_from_sessions(sessionized, review_cat_code): - sessionized["relevant"] = ( - (sessionized.tstamp_inSec - sessionized.session_id) - <= q08_SECONDS_BEFORE_PURCHASE - ) & (sessionized.wcs_sales_sk != NA_FLAG) - unique_sales_sk = ( - sessionized.query(f"wcs_sales_sk != {NA_FLAG}") - .query("relevant == True") - .query(f"wp_type_codes != {review_cat_code}") - .wcs_sales_sk.unique() - ) - - return unique_sales_sk - - -def prep_for_sessionization(df, review_cat_code): - df = df.fillna(NA_FLAG) - df = df.sort_values( - by=["wcs_user_sk", "tstamp_inSec", "wcs_sales_sk", "wp_type_codes"] - ).reset_index(drop=True) - - review_df = df.loc[df["wp_type_codes"] == review_cat_code] - # per user, the index of the first review - # need this to decide if a review was "recent enough" - every_users_first_review = ( - review_df[["wcs_user_sk", "tstamp_inSec"]] - .drop_duplicates() - .reset_index() - .groupby("wcs_user_sk")["index"] - .min() - .reset_index() - ) - every_users_first_review.columns = ["wcs_user_sk", "first_review_index"] - - # then reset the index to keep the old index before parallel join - df_merged = df.reset_index().merge( - every_users_first_review, how="left", on="wcs_user_sk" - ) - df_filtered = df_merged.query("index >= first_review_index") - return df_filtered - def read_tables(data_dir, c, config): table_reader = build_reader( @@ -235,7 +134,7 @@ def main(data_dir, client, c, config): c.drop_table("web_page_2") del web_page_df - + merged_df = merged_df.shuffle(on=["wcs_user_sk"]) merged_df["review_flag"] = merged_df.wp_type_codes == REVIEW_CAT_CODE diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18.py b/gpu_bdb/queries/q18/gpu_bdb_query_18.py index 899d1c86..55022f6b 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -14,7 +14,6 @@ # limitations under the License. # -import sys import os from collections import OrderedDict @@ -31,8 +30,8 @@ create_sentences_from_reviews, create_words_from_sentences, ) +from bdb_tools.q18_utils import find_relevant_reviews import numpy as np -import cupy as cp from distributed import wait @@ -77,81 +76,6 @@ def read_tables(config): return store_sales, date_dim, store, product_reviews -def create_found_reshaped_with_global_pos(found, targets): - """Given the dataframe created by mapping find_targets_in_reviews, - create a new dataframe in which the nonzero values in each row are exploded - to get their own row. Each row will contain the word, its mapping in the column order, - and the pr_review_sk for the review from which it came. - - Having these as two separate functions makes managing dask metadata easier. - """ - import cudf - - target_df = cudf.DataFrame({"word": targets}).reset_index(drop=False) - target_df.columns = ["word_mapping", "word"] - - df_clean = found.drop(["pr_review_sk"], axis=1) - - row_idxs, col_idxs = df_clean.values.nonzero() - - found_reshaped = cudf.DataFrame( - {"word_mapping": col_idxs, "pr_review_sk": found["pr_review_sk"].iloc[row_idxs]} - ) - found_reshaped = found_reshaped.merge(target_df, on="word_mapping", how="inner")[ - ["word", "pr_review_sk"] - ] - return found_reshaped - - -def find_targets_in_reviews_helper(ddf, targets, str_col_name="pr_review_content"): - """returns a N x K matrix, where N is the number of rows in ddf that - contain one of the target words and K is the number of words in targets. - - If a target word is found in a review, the value in that row, column - is non-zero. - - At the end, any row with non-zero values is returned. - - """ - import cudf - from cudf._lib.strings import find_multiple - - lowered = ddf[str_col_name].str.lower() - - ## TODO: Do the replace/any in cupy land before going to cuDF - resdf = cudf.DataFrame( - cp.asarray( - find_multiple.find_multiple(lowered._column, targets._column) - ).reshape(-1, len(targets)) - ) - - resdf = resdf.replace([0, -1], [1, 0]) - found_mask = resdf.any(axis=1) - resdf["pr_review_sk"] = ddf["pr_review_sk"] - found = resdf.loc[found_mask] - return create_found_reshaped_with_global_pos(found, targets) - - -def find_relevant_reviews(df, targets, str_col_name="pr_review_content"): - """ - This function finds the reviews containg target stores and returns the - relevant reviews - """ - import cudf - - targets = cudf.Series(targets) - targets_lower = targets.str.lower() - reviews_found = find_targets_in_reviews_helper(df, targets_lower)[ - ["word", "pr_review_sk"] - ] - - combined = reviews_found.merge( - df[["pr_review_date", "pr_review_sk"]], how="inner", on=["pr_review_sk"] - ) - - return combined - - def main(client, config): import cudf import dask_cudf diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index 3d15cf2e..84f58d7e 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -14,12 +14,10 @@ # limitations under the License. # -import sys import os from bdb_tools.cluster_startup import attach_to_cluster import numpy as np -import cupy as cp import dask_cudf @@ -33,6 +31,8 @@ from bdb_tools.readers import build_reader +from bdb_tools.q18_utils import find_relevant_reviews + from dask.distributed import wait @@ -44,81 +44,6 @@ EOL_CHAR = "รจ" -def create_found_reshaped_with_global_pos(found, targets): - """Given the dataframe created by mapping find_targets_in_reviews, - create a new dataframe in which the nonzero values in each row are exploded - to get their own row. Each row will contain the word, its mapping in the column order, - and the pr_review_sk for the review from which it came. - - Having these as two separate functions makes managing dask metadata easier. - """ - import cudf - - target_df = cudf.DataFrame({"word": targets}).reset_index(drop=False) - target_df.columns = ["word_mapping", "word"] - - df_clean = found.drop(["pr_review_sk"], axis=1) - - row_idxs, col_idxs = df_clean.values.nonzero() - - found_reshaped = cudf.DataFrame( - {"word_mapping": col_idxs, "pr_review_sk": found["pr_review_sk"].iloc[row_idxs]} - ) - found_reshaped = found_reshaped.merge(target_df, on="word_mapping", how="inner")[ - ["word", "pr_review_sk"] - ] - return found_reshaped - - -def find_targets_in_reviews_helper(ddf, targets, str_col_name="pr_review_content"): - """returns a N x K matrix, where N is the number of rows in ddf that - contain one of the target words and K is the number of words in targets. - - If a target word is found in a review, the value in that row, column - is non-zero. - - At the end, any row with non-zero values is returned. - - """ - import cudf - from cudf._lib.strings import find_multiple - - lowered = ddf[str_col_name].str.lower() - - ## TODO: Do the replace/any in cupy land before going to cuDF - resdf = cudf.DataFrame( - cp.asarray( - find_multiple.find_multiple(lowered._column, targets._column) - ).reshape(-1, len(targets)) - ) - - resdf = resdf.replace([0, -1], [1, 0]) - found_mask = resdf.any(axis=1) - resdf["pr_review_sk"] = ddf["pr_review_sk"] - found = resdf.loc[found_mask] - return create_found_reshaped_with_global_pos(found, targets) - - -def find_relevant_reviews(df, targets, str_col_name="pr_review_content"): - """ - This function finds the reviews containg target stores and returns the - relevant reviews - """ - import cudf - - targets = cudf.Series(targets) - targets_lower = targets.str.lower() - reviews_found = find_targets_in_reviews_helper(df, targets_lower)[ - ["word", "pr_review_sk"] - ] - - combined = reviews_found.merge( - df[["pr_review_date", "pr_review_sk"]], how="inner", on=["pr_review_sk"] - ) - - return combined - - def read_tables(data_dir, c, config): table_reader = build_reader( data_format=config["file_format"], basepath=config["data_dir"], diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20.py b/gpu_bdb/queries/q20/gpu_bdb_query_20.py index 8db6d19d..e7351885 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -14,12 +14,8 @@ # limitations under the License. # -import sys -import cupy as cp -import rmm import numpy as np - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -27,16 +23,11 @@ run_query, ) from bdb_tools.readers import build_reader +from bdb_tools.q20_utils import get_clusters from dask import delayed from dask.distributed import wait -# q20 parameters -N_CLUSTERS = 8 -CLUSTER_ITERATIONS = 20 -N_ITER = 5 - - def read_tables(config): table_reader = build_reader( data_format=config["file_format"], @@ -64,31 +55,6 @@ def read_tables(config): return store_sales_df, store_returns_df -def get_clusters(client, ml_input_df, feature_cols): - """ - Takes the dask client, kmeans_input_df and feature columns. - Returns a dictionary matching the output required for q20 - """ - import dask_cudf - - ml_tasks = [ - delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) - for df in ml_input_df[feature_cols].to_delayed() - ] - - results_dict = client.compute(*ml_tasks, sync=True) - - labels = results_dict["cid_labels"] - - labels_final = dask_cudf.from_cudf(labels, npartitions=ml_input_df.npartitions) - ml_input_df["label"] = labels_final.reset_index()[0] - - output = ml_input_df[["user_sk", "label"]] - - results_dict["cid_labels"] = output - return results_dict - - def remove_inf_and_nulls(df, column_names, value=0.0): """ Replace all nulls, inf, -inf with value column_name from df diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index db8a54eb..58f1ab94 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -26,41 +26,13 @@ benchmark, gpubdb_argparser, run_query, - train_clustering_model ) from bdb_tools.readers import build_reader -from dask_sql import Context - -# q20 parameters -N_CLUSTERS = 8 -CLUSTER_ITERATIONS = 20 -N_ITER = 5 - - -def get_clusters(client, ml_input_df, feature_cols): - """ - Takes the dask client, kmeans_input_df and feature columns. - Returns a dictionary matching the output required for q20 - """ - import dask_cudf - ml_tasks = [ - delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) - for df in ml_input_df[feature_cols].to_delayed() - ] +from bdb_tools.q20_utils import get_clusters - results_dict = client.compute(*ml_tasks, sync=True) - - labels = results_dict["cid_labels"] - - labels_final = dask_cudf.from_cudf(labels, npartitions=ml_input_df.npartitions) - ml_input_df["label"] = labels_final.reset_index()[0] - - output = ml_input_df[["user_sk", "label"]] - - results_dict["cid_labels"] = output - return results_dict +from dask_sql import Context def read_tables(data_dir, c, config): diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27.py b/gpu_bdb/queries/q27/gpu_bdb_query_27.py index 25eb247e..20da67a6 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -14,26 +14,26 @@ # limitations under the License. # -import sys import time import argparse -import spacy -import rmm -import cupy as cp -import distributed - from bdb_tools.utils import ( benchmark, gpubdb_argparser, left_semi_join, - run_query, + run_query +) + +from bdb_tools.text import ( + create_sentences_from_reviews, + create_words_from_sentences ) -from bdb_tools.text import create_sentences_from_reviews, create_words_from_sentences from bdb_tools.readers import build_reader -from dask_cuda import LocalCUDACluster -from dask.distributed import Client, wait + +from bdb_tools.q27_utils import ner_parser + +from dask.distributed import wait # -------- Q27 ----------- @@ -55,20 +55,6 @@ def read_tables(config): return product_reviews_df -def ner_parser(df, col_string, batch_size=256): - spacy.require_gpu() - nlp = spacy.load("en_core_web_sm") - docs = nlp.pipe(df[col_string], disable=["tagger", "parser"], batch_size=batch_size) - out = [] - for doc in docs: - l = [ent.text for ent in doc.ents if ent.label_ == "ORG"] - val = ", " - l = val.join(l) - out.append(l) - df["company_name_list"] = out - return df - - def main(client, config): import dask_cudf diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index 9c1b8046..a707d4c7 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.text import ( create_sentences_from_reviews, create_words_from_sentences @@ -24,7 +21,6 @@ from bdb_tools.cluster_startup import attach_to_cluster from dask.distributed import wait -import spacy from bdb_tools.utils import ( benchmark, @@ -34,6 +30,8 @@ from bdb_tools.readers import build_reader +from bdb_tools.q27_utils import ner_parser + from dask.distributed import wait @@ -57,20 +55,6 @@ def read_tables(data_dir, c, config): c.create_table("product_reviews", product_reviews_df, persist=False) -def ner_parser(df, col_string, batch_size=256): - spacy.require_gpu() - nlp = spacy.load("en_core_web_sm") - docs = nlp.pipe(df[col_string], disable=["tagger", "parser"], batch_size=batch_size) - out = [] - for doc in docs: - l = [ent.text for ent in doc.ents if ent.label_ == "ORG"] - val = ", " - l = val.join(l) - out.append(l) - df["company_name_list"] = out - return df - - def main(data_dir, client, c, config): benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28.py b/gpu_bdb/queries/q28/gpu_bdb_query_28.py index 8ecdf712..4adf3cc3 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -35,59 +35,7 @@ ) from bdb_tools.readers import build_reader - -QUERY_NUM = os.getcwd().split("/")[-1][1:] - -N_FEATURES = 2 ** 23 # Spark is doing 2^20 -ngram_range = (1, 2) -preprocessor = lambda s:s.str.lower() -norm = None -alternate_sign = False - - -def gpu_hashing_vectorizer(x): - vec = HashingVectorizer(n_features=N_FEATURES, - alternate_sign=alternate_sign, - ngram_range=ngram_range, - norm=norm, - preprocessor=preprocessor - ) - return vec.fit_transform(x) - - -def map_labels(ser): - import cudf - output_ser = cudf.Series(cudf.core.column.full(size=len(ser), fill_value=2, dtype=np.int32)) - zero_flag = (ser==1) | (ser==2) - output_ser.loc[zero_flag]=0 - - three_flag = (ser==3) - output_ser.loc[three_flag]=1 - - return output_ser - - -def build_features(t): - X = t["pr_review_content"] - X = X.map_partitions( - gpu_hashing_vectorizer, - meta=dask.array.from_array( - cupy.sparse.csr_matrix(cupy.zeros(1, dtype=cp.float32)) - ), - ) - - X = X.astype(np.float32).persist() - X.compute_chunk_sizes() - - return X - - -def build_labels(reviews_df): - y = reviews_df["pr_review_rating"].map_partitions(map_labels) - y = y.map_partitions(lambda x: cupy.asarray(x, cupy.int32)).persist() - y.compute_chunk_sizes() - - return y +from bdb_tools.q28_utils import post_etl_processing def read_tables(config): @@ -107,217 +55,6 @@ def read_tables(config): return ret -def categoricalize(num_sr): - return num_sr.astype("str").str.replace(["0", "1", "2"], ["NEG", "NEUT", "POS"]) - - -def sum_tp_fp(y_y_pred, nclasses): - - y, y_pred = y_y_pred - res = cp.zeros((nclasses, 2), order="F") - - for i in range(nclasses): - pos_pred_ix = cp.where(y_pred == i)[0] - - # short circuit - if len(pos_pred_ix) == 0: - res[i] = 0 - break - - tp_sum = (y_pred[pos_pred_ix] == y[pos_pred_ix]).sum() - fp_sum = (y_pred[pos_pred_ix] != y[pos_pred_ix]).sum() - res[i][0] = tp_sum - res[i][1] = fp_sum - return res - - -def precision_score(client, y, y_pred, average="binary"): - from cuml.dask.common.input_utils import DistributedDataHandler - - nclasses = len(cp.unique(y.map_blocks(lambda x: cp.unique(x)).compute())) - - if average == "binary" and nclasses > 2: - raise ValueError - - if nclasses < 2: - raise ValueError("Single class precision is not yet supported") - - ddh = DistributedDataHandler.create([y, y_pred]) - - precision_scores = client.compute( - [ - client.submit(sum_tp_fp, part, nclasses, workers=[worker]) - for worker, part in ddh.gpu_futures - ], - sync=True, - ) - - res = cp.zeros((nclasses, 2), order="F") - - for i in precision_scores: - res += i - - if average == "binary" or average == "macro": - - prec = cp.zeros(nclasses) - for i in range(nclasses): - tp_sum, fp_sum = res[i] - prec[i] = (tp_sum / (tp_sum + fp_sum)).item() - - if average == "binary": - return prec[nclasses - 1].item() - else: - return prec.mean().item() - else: - global_tp = cp.sum(res[:, 0]) - global_fp = cp.sum(res[:, 1]) - - return global_tp / (global_tp + global_fp).item() - - -def local_cm(y_y_pred, unique_labels, sample_weight): - - y_true, y_pred = y_y_pred - labels = unique_labels - - n_labels = labels.size - - # Assume labels are monotonically increasing for now. - - # intersect y_pred, y_true with labels, eliminate items not in labels - ind = cp.logical_and(y_pred < n_labels, y_true < n_labels) - y_pred = y_pred[ind] - y_true = y_true[ind] - - if sample_weight is None: - sample_weight = cp.ones(y_true.shape[0], dtype=np.int64) - else: - sample_weight = cp.asarray(sample_weight) - - sample_weight = sample_weight[ind] - - cm = cp.sparse.coo_matrix( - (sample_weight, (y_true, y_pred)), shape=(n_labels, n_labels), dtype=cp.float32, - ).toarray() - - return cp.nan_to_num(cm) - - -def confusion_matrix(client, y_true, y_pred, normalize=None, sample_weight=None): - from cuml.dask.common.input_utils import DistributedDataHandler - - unique_classes = cp.unique(y_true.map_blocks(lambda x: cp.unique(x)).compute()) - nclasses = len(unique_classes) - - ddh = DistributedDataHandler.create([y_true, y_pred]) - - cms = client.compute( - [ - client.submit( - local_cm, part, unique_classes, sample_weight, workers=[worker] - ) - for worker, part in ddh.gpu_futures - ], - sync=True, - ) - - cm = cp.zeros((nclasses, nclasses)) - for i in cms: - cm += i - - with np.errstate(all="ignore"): - if normalize == "true": - cm = cm / cm.sum(axis=1, keepdims=True) - elif normalize == "pred": - cm = cm / cm.sum(axis=0, keepdims=True) - elif normalize == "all": - cm = cm / cm.sum() - cm = cp.nan_to_num(cm) - - return cm - - -def accuracy_score(client, y, y_hat): - from uuid import uuid1 - from cuml.dask.common.input_utils import DistributedDataHandler - - ddh = DistributedDataHandler.create([y_hat, y]) - - def _count_accurate_predictions(y_hat_y): - y_hat, y = y_hat_y - y_hat = cp.asarray(y_hat, dtype=y_hat.dtype) - y = cp.asarray(y, dtype=y.dtype) - return y.shape[0] - cp.count_nonzero(y - y_hat) - - key = uuid1() - - futures = client.compute( - [ - client.submit( - _count_accurate_predictions, - worker_future[1], - workers=[worker_future[0]], - key="%s-%s" % (key, idx), - ) - for idx, worker_future in enumerate(ddh.gpu_futures) - ], - sync=True, - ) - - return sum(futures) / y.shape[0] - - -def post_etl_processing(client, train_data, test_data): - import cudf - from cuml.dask.naive_bayes import MultinomialNB as DistMNB - from cuml.dask.common import to_dask_cudf - from cuml.dask.common.input_utils import DistributedDataHandler - - # Feature engineering - X_train = build_features(train_data) - X_test = build_features(test_data) - - y_train = build_labels(train_data) - y_test = build_labels(test_data) - - # Perform ML - model = DistMNB(client=client, alpha=0.001) - model.fit(X_train, y_train) - - ### this regression seems to be coming from here - test_pred_st = time.time() - y_hat = model.predict(X_test).persist() - - # Compute distributed performance metrics - acc = accuracy_score(client, y_test, y_hat) - - print("Accuracy: " + str(acc)) - prec = precision_score(client, y_test, y_hat, average="macro") - - print("Precision: " + str(prec)) - cmat = confusion_matrix(client, y_test, y_hat) - - print("Confusion Matrix: " + str(cmat)) - metric_et = time.time() - - # Place results back in original Dataframe - - ddh = DistributedDataHandler.create(y_hat) - test_preds = to_dask_cudf( - [client.submit(cudf.Series, part) for w, part in ddh.gpu_futures] - ) - - test_preds = test_preds.map_partitions(categoricalize) - - test_data["prediction"] = test_preds - - final_data = test_data[["pr_review_sk", "pr_review_rating", "prediction"]].persist() - - final_data = final_data.sort_values("pr_review_sk").reset_index(drop=True) - wait(final_data) - return final_data, acc, prec, cmat - - def main(client, config): q_st = time.time() product_reviews_df = benchmark( diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 77966dd4..4b1bd773 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -14,17 +14,7 @@ # limitations under the License. # -import sys -import os -import time - from bdb_tools.cluster_startup import attach_to_cluster -from cuml.feature_extraction.text import HashingVectorizer -import cupy -import dask -from distributed import wait -import cupy as cp -import numpy as np from bdb_tools.utils import ( benchmark, @@ -34,268 +24,7 @@ from bdb_tools.readers import build_reader - -N_FEATURES = 2 ** 23 # Spark is doing 2^20 -ngram_range = (1, 2) -preprocessor = lambda s:s.str.lower() -norm = None -alternate_sign = False - - -def gpu_hashing_vectorizer(x): - vec = HashingVectorizer(n_features=N_FEATURES, - alternate_sign=alternate_sign, - ngram_range=ngram_range, - norm=norm, - preprocessor=preprocessor - ) - return vec.fit_transform(x) - - -def map_labels(ser): - import cudf - output_ser = cudf.Series(cudf.core.column.full(size=len(ser), fill_value=2, dtype=np.int32)) - zero_flag = (ser==1) | (ser==2) - output_ser.loc[zero_flag]=0 - - three_flag = (ser==3) - output_ser.loc[three_flag]=1 - - return output_ser - - -def build_features(t): - X = t["pr_review_content"] - X = X.map_partitions( - gpu_hashing_vectorizer, - meta=dask.array.from_array( - cupy.sparse.csr_matrix(cupy.zeros(1, dtype=cp.float32)) - ), - ) - - X = X.astype(np.float32).persist() - X.compute_chunk_sizes() - - return X - - -def build_labels(reviews_df): - y = reviews_df["pr_review_rating"].map_partitions(map_labels) - y = y.map_partitions(lambda x: cupy.asarray(x, cupy.int32)).persist() - y.compute_chunk_sizes() - - return y - - -def categoricalize(num_sr): - return num_sr.astype("str").str.replace(["0", "1", "2"], ["NEG", "NEUT", "POS"]) - - -def sum_tp_fp(y_y_pred, nclasses): - - y, y_pred = y_y_pred - res = cp.zeros((nclasses, 2), order="F") - - for i in range(nclasses): - pos_pred_ix = cp.where(y_pred == i)[0] - - # short circuit - if len(pos_pred_ix) == 0: - res[i] = 0 - break - - tp_sum = (y_pred[pos_pred_ix] == y[pos_pred_ix]).sum() - fp_sum = (y_pred[pos_pred_ix] != y[pos_pred_ix]).sum() - res[i][0] = tp_sum - res[i][1] = fp_sum - return res - - -def precision_score(client, y, y_pred, average="binary"): - from cuml.dask.common.input_utils import DistributedDataHandler - - nclasses = len(cp.unique(y.map_blocks(lambda x: cp.unique(x)).compute())) - - if average == "binary" and nclasses > 2: - raise ValueError - - if nclasses < 2: - raise ValueError("Single class precision is not yet supported") - - ddh = DistributedDataHandler.create([y, y_pred]) - - precision_scores = client.compute( - [ - client.submit(sum_tp_fp, part, nclasses, workers=[worker]) - for worker, part in ddh.gpu_futures - ], - sync=True, - ) - - res = cp.zeros((nclasses, 2), order="F") - - for i in precision_scores: - res += i - - if average == "binary" or average == "macro": - - prec = cp.zeros(nclasses) - for i in range(nclasses): - tp_sum, fp_sum = res[i] - prec[i] = (tp_sum / (tp_sum + fp_sum)).item() - - if average == "binary": - return prec[nclasses - 1].item() - else: - return prec.mean().item() - else: - global_tp = cp.sum(res[:, 0]) - global_fp = cp.sum(res[:, 1]) - - return global_tp / (global_tp + global_fp).item() - - -def local_cm(y_y_pred, unique_labels, sample_weight): - - y_true, y_pred = y_y_pred - labels = unique_labels - - n_labels = labels.size - - # Assume labels are monotonically increasing for now. - - # intersect y_pred, y_true with labels, eliminate items not in labels - ind = cp.logical_and(y_pred < n_labels, y_true < n_labels) - y_pred = y_pred[ind] - y_true = y_true[ind] - - if sample_weight is None: - sample_weight = cp.ones(y_true.shape[0], dtype=np.int64) - else: - sample_weight = cp.asarray(sample_weight) - - sample_weight = sample_weight[ind] - - cm = cp.sparse.coo_matrix( - (sample_weight, (y_true, y_pred)), shape=(n_labels, n_labels), dtype=cp.float32, - ).toarray() - - return cp.nan_to_num(cm) - - -def confusion_matrix(client, y_true, y_pred, normalize=None, sample_weight=None): - from cuml.dask.common.input_utils import DistributedDataHandler - - unique_classes = cp.unique(y_true.map_blocks(lambda x: cp.unique(x)).compute()) - nclasses = len(unique_classes) - - ddh = DistributedDataHandler.create([y_true, y_pred]) - - cms = client.compute( - [ - client.submit( - local_cm, part, unique_classes, sample_weight, workers=[worker] - ) - for worker, part in ddh.gpu_futures - ], - sync=True, - ) - - cm = cp.zeros((nclasses, nclasses)) - for i in cms: - cm += i - - with np.errstate(all="ignore"): - if normalize == "true": - cm = cm / cm.sum(axis=1, keepdims=True) - elif normalize == "pred": - cm = cm / cm.sum(axis=0, keepdims=True) - elif normalize == "all": - cm = cm / cm.sum() - cm = cp.nan_to_num(cm) - - return cm - - -def accuracy_score(client, y, y_hat): - from uuid import uuid1 - from cuml.dask.common.input_utils import DistributedDataHandler - - ddh = DistributedDataHandler.create([y_hat, y]) - - def _count_accurate_predictions(y_hat_y): - y_hat, y = y_hat_y - y_hat = cp.asarray(y_hat, dtype=y_hat.dtype) - y = cp.asarray(y, dtype=y.dtype) - return y.shape[0] - cp.count_nonzero(y - y_hat) - - key = uuid1() - - futures = client.compute( - [ - client.submit( - _count_accurate_predictions, - worker_future[1], - workers=[worker_future[0]], - key="%s-%s" % (key, idx), - ) - for idx, worker_future in enumerate(ddh.gpu_futures) - ], - sync=True, - ) - - return sum(futures) / y.shape[0] - - -def post_etl_processing(client, train_data, test_data): - import cudf - from cuml.dask.naive_bayes import MultinomialNB as DistMNB - from cuml.dask.common import to_dask_cudf - from cuml.dask.common.input_utils import DistributedDataHandler - - # Feature engineering - X_train = build_features(train_data) - X_test = build_features(test_data) - - y_train = build_labels(train_data) - y_test = build_labels(test_data) - - # Perform ML - model = DistMNB(client=client, alpha=0.001) - model.fit(X_train, y_train) - - ### this regression seems to be coming from here - test_pred_st = time.time() - y_hat = model.predict(X_test).persist() - - # Compute distributed performance metrics - acc = accuracy_score(client, y_test, y_hat) - - print("Accuracy: " + str(acc)) - prec = precision_score(client, y_test, y_hat, average="macro") - - print("Precision: " + str(prec)) - cmat = confusion_matrix(client, y_test, y_hat) - - print("Confusion Matrix: " + str(cmat)) - metric_et = time.time() - - # Place results back in original Dataframe - - ddh = DistributedDataHandler.create(y_hat) - test_preds = to_dask_cudf( - [client.submit(cudf.Series, part) for w, part in ddh.gpu_futures] - ) - - test_preds = test_preds.map_partitions(categoricalize) - - test_data["prediction"] = test_preds - - final_data = test_data[["pr_review_sk", "pr_review_rating", "prediction"]].persist() - - final_data = final_data.sort_values("pr_review_sk").reset_index(drop=True) - wait(final_data) - return final_data, acc, prec, cmat +from bdb_tools.q28_utils import post_etl_processing def read_tables(data_dir, c, config): From 999818b7450fb86de75415c3ff6232a2d730ed7e Mon Sep 17 00:00:00 2001 From: sft-managed Date: Tue, 11 Jan 2022 16:01:19 +0000 Subject: [PATCH 34/51] Remove lock files --- gpu_bdb/queries/q20/dask-worker-space/global.lock | 0 gpu_bdb/queries/q20/dask-worker-space/purge.lock | 0 2 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 gpu_bdb/queries/q20/dask-worker-space/global.lock delete mode 100644 gpu_bdb/queries/q20/dask-worker-space/purge.lock diff --git a/gpu_bdb/queries/q20/dask-worker-space/global.lock b/gpu_bdb/queries/q20/dask-worker-space/global.lock deleted file mode 100644 index e69de29b..00000000 diff --git a/gpu_bdb/queries/q20/dask-worker-space/purge.lock b/gpu_bdb/queries/q20/dask-worker-space/purge.lock deleted file mode 100644 index e69de29b..00000000 From 10c71d1b361787c9a336209e36e6c9021a379488 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Wed, 19 Jan 2022 04:22:48 +0000 Subject: [PATCH 35/51] Remove category codes casts --- gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 8 +++----- gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 8 +------- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index 785a5c2e..ad33cc0e 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -17,6 +17,8 @@ import sys import os +import cudf + from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( @@ -77,11 +79,7 @@ def main(data_dir, client, c, config): ORDER_CAT_CODE = cpu_categories.get_loc("order") # ### cast to minimum viable dtype - import cudf - codes_min_signed_type = cudf.utils.dtypes.min_signed_type( - len(cpu_categories)) - wp["wp_type_codes"] = wp["wp_type"].cat.codes.astype(codes_min_signed_type) - wp["wp_type"] = wp["wp_type"].cat.codes.astype(codes_min_signed_type) + wp["wp_type_codes"] = wp["wp_type"].cat.codes cols_2_keep = ["wp_web_page_sk", "wp_type_codes"] wp = wp[cols_2_keep] diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 634c1305..f15d8229 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -16,8 +16,6 @@ import os -import cudf - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( @@ -104,11 +102,7 @@ def main(data_dir, client, c, config): cpu_categories = web_page_df["wp_type"].compute().cat.categories.to_pandas() REVIEW_CAT_CODE = cpu_categories.get_loc("review") - codes_min_signed_type = cudf.utils.dtypes.min_signed_type(len(cpu_categories)) - - web_page_df["wp_type_codes"] = web_page_df["wp_type"].cat.codes.astype( - codes_min_signed_type - ) + web_page_df["wp_type_codes"] = web_page_df["wp_type"].cat.codes web_page_newcols = ["wp_web_page_sk", "wp_type_codes"] web_page_df = web_page_df[web_page_newcols] From 8d9db0dc16d57c6ee7bbd6c82cb8c4d90d5e6e2a Mon Sep 17 00:00:00 2001 From: sft-managed Date: Thu, 20 Jan 2022 06:11:38 +0000 Subject: [PATCH 36/51] Refactor read_tables and constants into shared files --- gpu_bdb/bdb_tools/q01_utils.py | 45 +++++++++ gpu_bdb/bdb_tools/q02_utils.py | 38 ++++++++ gpu_bdb/bdb_tools/q03_utils.py | 30 +++++- gpu_bdb/bdb_tools/q04_utils.py | 29 ++++++ gpu_bdb/bdb_tools/q05_utils.py | 32 +++++++ gpu_bdb/bdb_tools/q06_utils.py | 72 ++++++++++++++ gpu_bdb/bdb_tools/q07_utils.py | 55 +++++++++++ gpu_bdb/bdb_tools/q08_utils.py | 35 +++++++ gpu_bdb/bdb_tools/q09_utils.py | 88 ++++++++++++++++++ gpu_bdb/bdb_tools/q10_utils.py | 39 ++++++++ gpu_bdb/bdb_tools/q11_utils.py | 49 ++++++++++ gpu_bdb/bdb_tools/q12_utils.py | 44 +++++++++ gpu_bdb/bdb_tools/q13_utils.py | 45 +++++++++ gpu_bdb/bdb_tools/q14_utils.py | 47 ++++++++++ gpu_bdb/bdb_tools/q15_utils.py | 47 ++++++++++ gpu_bdb/bdb_tools/q16_utils.py | 52 +++++++++++ gpu_bdb/bdb_tools/q17_utils.py | 74 +++++++++++++++ gpu_bdb/bdb_tools/q18_utils.py | 47 ++++++++++ gpu_bdb/bdb_tools/q19_utils.py | 57 ++++++++++++ gpu_bdb/bdb_tools/q20_utils.py | 34 +++++++ gpu_bdb/bdb_tools/q21_utils.py | 69 ++++++++++++++ gpu_bdb/bdb_tools/q22_utils.py | 56 +++++++++++ gpu_bdb/bdb_tools/q23_utils.py | 45 +++++++++ gpu_bdb/bdb_tools/q24_utils.py | 49 ++++++++++ gpu_bdb/bdb_tools/q25_utils.py | 80 ++++++++++++++++ gpu_bdb/bdb_tools/q26_utils.py | 42 +++++++++ gpu_bdb/bdb_tools/q27_utils.py | 23 +++++ gpu_bdb/bdb_tools/q28_utils.py | 21 +++++ gpu_bdb/bdb_tools/q29_utils.py | 38 ++++++++ gpu_bdb/bdb_tools/q30_utils.py | 44 +++++++++ gpu_bdb/queries/q01/gpu_bdb_query_01.py | 31 ++----- .../queries/q01/gpu_bdb_query_01_dask_sql.py | 33 ++----- gpu_bdb/queries/q02/gpu_bdb_query_02.py | 21 ++--- .../queries/q02/gpu_bdb_query_02_dask_sql.py | 27 ++---- gpu_bdb/queries/q03/gpu_bdb_query_03.py | 25 ++--- .../queries/q03/gpu_bdb_query_03_dask_sql.py | 39 ++------ gpu_bdb/queries/q04/gpu_bdb_query_04.py | 26 +----- .../queries/q04/gpu_bdb_query_04_dask_sql.py | 28 +----- gpu_bdb/queries/q05/gpu_bdb_query_05.py | 27 +----- .../queries/q05/gpu_bdb_query_05_dask_sql.py | 37 +------- gpu_bdb/queries/q06/gpu_bdb_query_06.py | 54 ++--------- .../queries/q06/gpu_bdb_query_06_dask_sql.py | 59 ++---------- gpu_bdb/queries/q07/gpu_bdb_query_07.py | 35 +------ .../queries/q07/gpu_bdb_query_07_dask_sql.py | 33 +------ gpu_bdb/queries/q08/gpu_bdb_query_08.py | 26 +----- .../queries/q08/gpu_bdb_query_08_dask_sql.py | 40 +------- gpu_bdb/queries/q09/gpu_bdb_query_09.py | 88 +++++------------- .../queries/q09/gpu_bdb_query_09_dask_sql.py | 93 ++++++------------- gpu_bdb/queries/q10/gpu_bdb_query_10.py | 28 +----- .../queries/q10/gpu_bdb_query_10_dask_sql.py | 28 ++---- gpu_bdb/queries/q11/gpu_bdb_query_11.py | 33 +------ .../queries/q11/gpu_bdb_query_11_dask_sql.py | 33 +------ gpu_bdb/queries/q12/gpu_bdb_query_12.py | 20 +--- .../queries/q12/gpu_bdb_query_12_dask_sql.py | 25 +---- gpu_bdb/queries/q13/gpu_bdb_query_13.py | 24 +---- .../queries/q13/gpu_bdb_query_13_dask_sql.py | 30 +----- gpu_bdb/queries/q14/gpu_bdb_query_14.py | 26 +----- .../queries/q14/gpu_bdb_query_14_dask_sql.py | 30 +----- gpu_bdb/queries/q15/gpu_bdb_query_15.py | 31 ++----- .../queries/q15/gpu_bdb_query_15_dask_sql.py | 36 ++----- gpu_bdb/queries/q16/gpu_bdb_query_16.py | 31 +------ .../queries/q16/gpu_bdb_query_16_dask_sql.py | 35 +------ gpu_bdb/queries/q17/gpu_bdb_query_17.py | 59 ++---------- .../queries/q17/gpu_bdb_query_17_dask_sql.py | 54 ++--------- gpu_bdb/queries/q18/gpu_bdb_query_18.py | 50 ++-------- .../queries/q18/gpu_bdb_query_18_dask_sql.py | 54 ++--------- gpu_bdb/queries/q19/gpu_bdb_query_19.py | 37 +------- .../queries/q19/gpu_bdb_query_19_dask_sql.py | 43 ++------- gpu_bdb/queries/q20/gpu_bdb_query_20.py | 33 +------ .../queries/q20/gpu_bdb_query_20_dask_sql.py | 39 +------- gpu_bdb/queries/q21/gpu_bdb_query_21.py | 54 +---------- .../queries/q21/gpu_bdb_query_21_dask_sql.py | 47 +--------- gpu_bdb/queries/q22/gpu_bdb_query_22.py | 38 ++------ .../queries/q22/gpu_bdb_query_22_dask_sql.py | 49 ++-------- gpu_bdb/queries/q23/gpu_bdb_query_23.py | 32 ++----- .../queries/q23/gpu_bdb_query_23_dask_sql.py | 36 ++----- gpu_bdb/queries/q24/gpu_bdb_query_24.py | 29 +----- .../queries/q24/gpu_bdb_query_24_dask_sql.py | 34 +------ gpu_bdb/queries/q25/gpu_bdb_query_25.py | 40 ++------ .../queries/q25/gpu_bdb_query_25_dask_sql.py | 46 ++------- gpu_bdb/queries/q26/gpu_bdb_query_26.py | 33 ++----- .../queries/q26/gpu_bdb_query_26_dask_sql.py | 42 +++------ gpu_bdb/queries/q27/gpu_bdb_query_27.py | 27 ++---- .../queries/q27/gpu_bdb_query_27_dask_sql.py | 30 ++---- gpu_bdb/queries/q28/gpu_bdb_query_28.py | 23 +---- .../queries/q28/gpu_bdb_query_28_dask_sql.py | 26 +----- gpu_bdb/queries/q29/gpu_bdb_query_29.py | 21 +---- .../queries/q29/gpu_bdb_query_29_dask_sql.py | 26 ++---- gpu_bdb/queries/q30/gpu_bdb_query_30.py | 24 +---- .../queries/q30/gpu_bdb_query_30_dask_sql.py | 34 ++----- 90 files changed, 1771 insertions(+), 1847 deletions(-) create mode 100644 gpu_bdb/bdb_tools/q01_utils.py create mode 100644 gpu_bdb/bdb_tools/q02_utils.py create mode 100644 gpu_bdb/bdb_tools/q06_utils.py create mode 100644 gpu_bdb/bdb_tools/q07_utils.py create mode 100644 gpu_bdb/bdb_tools/q09_utils.py create mode 100644 gpu_bdb/bdb_tools/q10_utils.py create mode 100644 gpu_bdb/bdb_tools/q11_utils.py create mode 100644 gpu_bdb/bdb_tools/q12_utils.py create mode 100644 gpu_bdb/bdb_tools/q13_utils.py create mode 100644 gpu_bdb/bdb_tools/q14_utils.py create mode 100644 gpu_bdb/bdb_tools/q15_utils.py create mode 100644 gpu_bdb/bdb_tools/q16_utils.py create mode 100644 gpu_bdb/bdb_tools/q17_utils.py create mode 100644 gpu_bdb/bdb_tools/q19_utils.py create mode 100644 gpu_bdb/bdb_tools/q21_utils.py create mode 100644 gpu_bdb/bdb_tools/q22_utils.py create mode 100644 gpu_bdb/bdb_tools/q23_utils.py create mode 100644 gpu_bdb/bdb_tools/q24_utils.py create mode 100644 gpu_bdb/bdb_tools/q25_utils.py create mode 100644 gpu_bdb/bdb_tools/q26_utils.py create mode 100644 gpu_bdb/bdb_tools/q29_utils.py create mode 100644 gpu_bdb/bdb_tools/q30_utils.py diff --git a/gpu_bdb/bdb_tools/q01_utils.py b/gpu_bdb/bdb_tools/q01_utils.py new file mode 100644 index 00000000..5212f4f3 --- /dev/null +++ b/gpu_bdb/bdb_tools/q01_utils.py @@ -0,0 +1,45 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +from bdb_tools.readers import build_reader + +# -------- Q1 ----------- +# -- sf1 -> 11 stores, 90k sales in 820k lines +q01_viewed_together_count = 50 +q01_limit = 100 + + +item_cols = ["i_item_sk", "i_category_id"] +ss_cols = ["ss_item_sk", "ss_store_sk", "ss_ticket_number"] + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_df = table_reader.read("item", relevant_cols=item_cols) + ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) + + if c: + c.create_table("item", item_df, persist=False) + c.create_table("store_sales", ss_df, persist=False) + + return item_df, ss_df + diff --git a/gpu_bdb/bdb_tools/q02_utils.py b/gpu_bdb/bdb_tools/q02_utils.py new file mode 100644 index 00000000..b9f058a6 --- /dev/null +++ b/gpu_bdb/bdb_tools/q02_utils.py @@ -0,0 +1,38 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +q02_item_sk = 10001 +q02_limit = 30 +q02_session_timeout_inSec = 3600 +q02_MAX_ITEMS_PER_BASKET = 5000000 + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + if c: + c.create_table("web_clickstreams", wcs_df, persist=False) + + return wcs_df + diff --git a/gpu_bdb/bdb_tools/q03_utils.py b/gpu_bdb/bdb_tools/q03_utils.py index 290b9d24..a02543c5 100644 --- a/gpu_bdb/bdb_tools/q03_utils.py +++ b/gpu_bdb/bdb_tools/q03_utils.py @@ -21,10 +21,38 @@ from numba import cuda -# -------- Q03 ----------- +from bdb_tools.readers import build_reader + q03_days_in_sec_before_purchase = 864000 q03_views_before_purchase = 5 q03_purchased_item_IN = 10001 +q03_limit = 100 + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_cols = ["i_category_id", "i_item_sk"] + wcs_cols = [ + "wcs_user_sk", + "wcs_click_time_sk", + "wcs_click_date_sk", + "wcs_item_sk", + "wcs_sales_sk", + ] + + item_df = table_reader.read("item", relevant_cols=item_cols) + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + if c: + c.create_table("web_clickstreams", wcs_df, persist=False) + c.create_table("item", item_df, persist=False) + + return item_df + @cuda.jit def find_items_viewed_before_purchase_kernel( diff --git a/gpu_bdb/bdb_tools/q04_utils.py b/gpu_bdb/bdb_tools/q04_utils.py index 70d75f2e..b848f840 100644 --- a/gpu_bdb/bdb_tools/q04_utils.py +++ b/gpu_bdb/bdb_tools/q04_utils.py @@ -18,6 +18,35 @@ from bdb_tools.sessionization import get_sessions +from bdb_tools.readers import build_reader + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + wp_cols = ["wp_type", "wp_web_page_sk"] + wp_df = table_reader.read("web_page", relevant_cols=wp_cols) + + wcs_cols = [ + "wcs_user_sk", + "wcs_click_date_sk", + "wcs_click_time_sk", + "wcs_web_page_sk", + "wcs_sales_sk", + ] + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + if c: + c.create_table('web_page_wo_categorical', wp_df, persist=False) + c.create_table('web_clickstreams', wcs_df, persist=False) + + return wp_df, wcs_df + + def abandonedShoppingCarts(df, DYNAMIC_CAT_CODE, ORDER_CAT_CODE): # Select groups where last dynamic row comes after last order row diff --git a/gpu_bdb/bdb_tools/q05_utils.py b/gpu_bdb/bdb_tools/q05_utils.py index c6ded04b..c4b71fd0 100644 --- a/gpu_bdb/bdb_tools/q05_utils.py +++ b/gpu_bdb/bdb_tools/q05_utils.py @@ -21,6 +21,8 @@ from bdb_tools.cupy_metrics import cupy_precision_score +from bdb_tools.readers import build_reader + from sklearn.metrics import roc_auc_score # Logistic Regression params @@ -31,6 +33,36 @@ C = 10_000 # reg_lambda = 0 hence C for model is a large value convergence_tol = 1e-9 +wcs_columns = ["wcs_item_sk", "wcs_user_sk"] +items_columns = ["i_item_sk", "i_category", "i_category_id"] +customer_columns = ["c_customer_sk", "c_current_cdemo_sk"] +customer_dem_columns = ["cd_demo_sk", "cd_gender", "cd_education_status"] + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_ddf = table_reader.read("item", relevant_cols=items_columns, index=False) + customer_ddf = table_reader.read( + "customer", relevant_cols=customer_columns, index=False + ) + customer_dem_ddf = table_reader.read( + "customer_demographics", relevant_cols=customer_dem_columns, index=False + ) + wcs_ddf = table_reader.read( + "web_clickstreams", relevant_cols=wcs_columns, index=False + ) + + if c: + c.create_table("web_clickstreams", wcs_ddf, persist=False) + c.create_table("customer", customer_ddf, persist=False) + c.create_table("item", item_ddf, persist=False) + c.create_table("customer_demographics", customer_dem_ddf, persist=False) + + return (item_ddf, customer_ddf, customer_dem_ddf) def build_and_predict_model(ml_input_df): """ diff --git a/gpu_bdb/bdb_tools/q06_utils.py b/gpu_bdb/bdb_tools/q06_utils.py new file mode 100644 index 00000000..ec4e02b3 --- /dev/null +++ b/gpu_bdb/bdb_tools/q06_utils.py @@ -0,0 +1,72 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +# -------- Q6 ----------- +q06_LIMIT = 100 +# --web_sales and store_sales date +q06_YEAR = 2001 + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + web_sales_cols = [ + "ws_bill_customer_sk", + "ws_sold_date_sk", + "ws_ext_list_price", + "ws_ext_wholesale_cost", + "ws_ext_discount_amt", + "ws_ext_sales_price", + ] + store_sales_cols = [ + "ss_customer_sk", + "ss_sold_date_sk", + "ss_ext_list_price", + "ss_ext_wholesale_cost", + "ss_ext_discount_amt", + "ss_ext_sales_price", + ] + date_cols = ["d_date_sk", "d_year", "d_moy"] + customer_cols = [ + "c_customer_sk", + "c_customer_id", + "c_email_address", + "c_first_name", + "c_last_name", + "c_preferred_cust_flag", + "c_birth_country", + "c_login", + ] + + ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) + ss_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_df = table_reader.read("date_dim", relevant_cols=date_cols) + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + + if c: + c.create_table('web_sales', ws_df, persist=False) + c.create_table('store_sales', ss_df, persist=False) + c.create_table('date_dim', date_df, persist=False) + c.create_table('customer', customer_df, persist=False) + + return (ws_df, ss_df, date_df, customer_df) + diff --git a/gpu_bdb/bdb_tools/q07_utils.py b/gpu_bdb/bdb_tools/q07_utils.py new file mode 100644 index 00000000..e55b54f1 --- /dev/null +++ b/gpu_bdb/bdb_tools/q07_utils.py @@ -0,0 +1,55 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_cols = ["i_item_sk", "i_current_price", "i_category"] + store_sales_cols = ["ss_item_sk", "ss_customer_sk", "ss_sold_date_sk"] + date_cols = ["d_date_sk", "d_year", "d_moy"] + customer_cols = ["c_customer_sk", "c_current_addr_sk"] + customer_address_cols = ["ca_address_sk", "ca_state"] + + item_df = table_reader.read("item", relevant_cols=item_cols) + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + customer_address_df = table_reader.read( + "customer_address", relevant_cols=customer_address_cols + ) + + if c: + c.create_table("item", item_df, persist=False) + c.create_table("customer", customer_df, persist=False) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("customer_address", customer_address_df, persist=False) + + return ( + item_df, + store_sales_df, + date_dim_df, + customer_df, + customer_address_df, + ) + + diff --git a/gpu_bdb/bdb_tools/q08_utils.py b/gpu_bdb/bdb_tools/q08_utils.py index 4766beb6..95a32cf8 100644 --- a/gpu_bdb/bdb_tools/q08_utils.py +++ b/gpu_bdb/bdb_tools/q08_utils.py @@ -19,9 +19,44 @@ import cupy as cp import numpy as np +from bdb_tools.readers import build_reader + +q08_STARTDATE = "2001-09-02" +q08_ENDDATE = "2002-09-02" q08_SECONDS_BEFORE_PURCHASE = 259200 NA_FLAG = 0 +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + date_dim_cols = ["d_date_sk", "d_date"] + web_page_cols = ["wp_web_page_sk", "wp_type"] + web_sales_cols = ["ws_net_paid", "ws_order_number", "ws_sold_date_sk"] + wcs_cols = [ + "wcs_user_sk", + "wcs_sales_sk", + "wcs_click_date_sk", + "wcs_click_time_sk", + "wcs_web_page_sk", + ] + + date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) + web_page_df = table_reader.read("web_page", relevant_cols=web_page_cols) + web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + if c: + c.create_table("web_clickstreams", wcs_df, persist=False) + c.create_table("web_sales", web_sales_df, persist=False) + c.create_table("web_page", web_page_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + + return (date_dim_df, web_page_df, web_sales_df) + def get_session_id_from_session_boundary(session_change_df, last_session_len): """ This function returns session starts given a session change df diff --git a/gpu_bdb/bdb_tools/q09_utils.py b/gpu_bdb/bdb_tools/q09_utils.py new file mode 100644 index 00000000..8ff86074 --- /dev/null +++ b/gpu_bdb/bdb_tools/q09_utils.py @@ -0,0 +1,88 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + + +# -------- Q9 ----------- +q09_year = 2001 + +q09_part1_ca_country = "United States" +q09_part1_net_profit_min = 0 +q09_part1_net_profit_max = 2000 +q09_part1_education_status = "4 yr Degree" +q09_part1_marital_status = "M" +q09_part1_sales_price_min = 100 +q09_part1_sales_price_max = 150 + +q09_part2_ca_country = "United States" +q09_part2_net_profit_min = 150 +q09_part2_net_profit_max = 3000 +q09_part2_education_status = "4 yr Degree" +q09_part2_marital_status = "M" +q09_part2_sales_price_min = 50 +q09_part2_sales_price_max = 200 + +q09_part3_ca_country = "United States" +q09_part3_net_profit_min = 50 +q09_part3_net_profit_max = 25000 +q09_part3_education_status = "4 yr Degree" +q09_part3_marital_status = "M" +q09_part3_sales_price_min = 150 +q09_part3_sales_price_max = 200 + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ss_columns = [ + "ss_quantity", + "ss_sold_date_sk", + "ss_addr_sk", + "ss_store_sk", + "ss_cdemo_sk", + "ss_sales_price", + "ss_net_profit", + ] + + store_sales = table_reader.read("store_sales", relevant_cols=ss_columns) + + ca_columns = ["ca_address_sk", "ca_country", "ca_state"] + customer_address = table_reader.read("customer_address", relevant_cols=ca_columns) + + cd_columns = ["cd_demo_sk", "cd_marital_status", "cd_education_status"] + customer_demographics = table_reader.read( + "customer_demographics", relevant_cols=cd_columns + ) + + dd_columns = ["d_year", "d_date_sk"] + date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) + + s_columns = ["s_store_sk"] + store = table_reader.read("store", relevant_cols=s_columns) + + if c: + c.create_table("store_sales", store_sales, persist=False) + c.create_table("customer_address", customer_address, persist=False) + c.create_table("customer_demographics", customer_demographics, persist=False) + c.create_table("date_dim", date_dim, persist=False) + c.create_table("store", store, persist=False) + + return (store_sales, customer_address, customer_demographics, date_dim, store) + diff --git a/gpu_bdb/bdb_tools/q10_utils.py b/gpu_bdb/bdb_tools/q10_utils.py new file mode 100644 index 00000000..938aa45f --- /dev/null +++ b/gpu_bdb/bdb_tools/q10_utils.py @@ -0,0 +1,39 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +eol_char = "รจ" + +def read_tables(config, c=None): + + ### splitting by row groups for better parallelism + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] + + product_reviews_df = table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols, + ) + + if c: + c.create_table("product_reviews", product_reviews_df, persist=False) + + return product_reviews_df + diff --git a/gpu_bdb/bdb_tools/q11_utils.py b/gpu_bdb/bdb_tools/q11_utils.py new file mode 100644 index 00000000..603d3d79 --- /dev/null +++ b/gpu_bdb/bdb_tools/q11_utils.py @@ -0,0 +1,49 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + product_review_cols = [ + "pr_review_rating", + "pr_item_sk", + ] + web_sales_cols = [ + "ws_sold_date_sk", + "ws_net_paid", + "ws_item_sk", + ] + date_cols = ["d_date_sk", "d_date"] + + pr_df = table_reader.read("product_reviews", relevant_cols=product_review_cols) + # we only read int columns here so it should scale up to sf-10k as just 26M rows + pr_df = pr_df.repartition(npartitions=1) + + ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) + date_df = table_reader.read("date_dim", relevant_cols=date_cols) + + if c: + c.create_table("web_sales", ws_df, persist=False) + c.create_table("product_reviews", pr_df, persist=False) + c.create_table("date_dim", date_df, persist=False) + + return (pr_df, ws_df, date_df) diff --git a/gpu_bdb/bdb_tools/q12_utils.py b/gpu_bdb/bdb_tools/q12_utils.py new file mode 100644 index 00000000..e1b72cd2 --- /dev/null +++ b/gpu_bdb/bdb_tools/q12_utils.py @@ -0,0 +1,44 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + + +q12_i_category_IN = "'Books', 'Electronics'" + +item_cols = ["i_item_sk", "i_category"] +store_sales_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_customer_sk"] +wcs_cols = ["wcs_user_sk", "wcs_click_date_sk", "wcs_item_sk", "wcs_sales_sk"] + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_df = table_reader.read("item", relevant_cols=item_cols) + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + if c: + c.create_table("web_clickstreams", wcs_df, persist=False) + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("item", item_df, persist=False) + + return item_df, store_sales_df + diff --git a/gpu_bdb/bdb_tools/q13_utils.py b/gpu_bdb/bdb_tools/q13_utils.py new file mode 100644 index 00000000..96910386 --- /dev/null +++ b/gpu_bdb/bdb_tools/q13_utils.py @@ -0,0 +1,45 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + date_cols = ["d_date_sk", "d_year"] + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + + customer_cols = ["c_customer_sk", "c_customer_id", "c_first_name", "c_last_name"] + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + + s_sales_cols = ["ss_sold_date_sk", "ss_customer_sk", "ss_net_paid"] + s_sales_df = table_reader.read("store_sales", relevant_cols=s_sales_cols) + + w_sales_cols = ["ws_sold_date_sk", "ws_bill_customer_sk", "ws_net_paid"] + web_sales_df = table_reader.read("web_sales", relevant_cols=w_sales_cols) + + if c: + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("customer", customer_df, persist=False) + c.create_table("store_sales", s_sales_df, persist=False) + c.create_table("web_sales", web_sales_df, persist=False) + + return (date_dim_df, customer_df, s_sales_df, web_sales_df) + diff --git a/gpu_bdb/bdb_tools/q14_utils.py b/gpu_bdb/bdb_tools/q14_utils.py new file mode 100644 index 00000000..b7c900b4 --- /dev/null +++ b/gpu_bdb/bdb_tools/q14_utils.py @@ -0,0 +1,47 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ws_columns = ["ws_ship_hdemo_sk", "ws_web_page_sk", "ws_sold_time_sk"] + web_sales = table_reader.read("web_sales", relevant_cols=ws_columns) + + hd_columns = ["hd_demo_sk", "hd_dep_count"] + household_demographics = table_reader.read( + "household_demographics", relevant_cols=hd_columns + ) + + wp_columns = ["wp_web_page_sk", "wp_char_count"] + web_page = table_reader.read("web_page", relevant_cols=wp_columns) + + td_columns = ["t_time_sk", "t_hour"] + time_dim = table_reader.read("time_dim", relevant_cols=td_columns) + + if c: + c.create_table("household_demographics", household_demographics, persist=False) + c.create_table("web_page", web_page, persist=False) + c.create_table("web_sales", web_sales, persist=False) + c.create_table("time_dim", time_dim, persist=False) + + return (web_sales, household_demographics, web_page, time_dim) + diff --git a/gpu_bdb/bdb_tools/q15_utils.py b/gpu_bdb/bdb_tools/q15_utils.py new file mode 100644 index 00000000..08f1f6d7 --- /dev/null +++ b/gpu_bdb/bdb_tools/q15_utils.py @@ -0,0 +1,47 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +# --store_sales date range +q15_startDate = "2001-09-02" +# --+1year +q15_endDate = "2002-09-02" +q15_store_sk = 10 + +store_sales_cols = ["ss_sold_date_sk", "ss_net_paid", "ss_store_sk", "ss_item_sk"] +date_cols = ["d_date", "d_date_sk"] +item_cols = ["i_item_sk", "i_category_id"] + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + + if c: + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("item", item_df, persist=False) + + return store_sales_df, date_dim_df, item_df + diff --git a/gpu_bdb/bdb_tools/q16_utils.py b/gpu_bdb/bdb_tools/q16_utils.py new file mode 100644 index 00000000..8631bb28 --- /dev/null +++ b/gpu_bdb/bdb_tools/q16_utils.py @@ -0,0 +1,52 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +websale_cols = [ + "ws_order_number", + "ws_item_sk", + "ws_warehouse_sk", + "ws_sold_date_sk", + "ws_sales_price", +] +web_returns_cols = ["wr_order_number", "wr_item_sk", "wr_refunded_cash"] +date_cols = ["d_date", "d_date_sk"] +item_cols = ["i_item_sk", "i_item_id"] +warehouse_cols = ["w_warehouse_sk", "w_state"] + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) + web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + warehouse_df = table_reader.read("warehouse", relevant_cols=warehouse_cols) + + if c: + c.create_table("web_sales", web_sales_df, persist=False) + c.create_table("web_returns", web_returns_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("warehouse", warehouse_df, persist=False) + + return web_sales_df, web_returns_df, date_dim_df, item_df, warehouse_df + diff --git a/gpu_bdb/bdb_tools/q17_utils.py b/gpu_bdb/bdb_tools/q17_utils.py new file mode 100644 index 00000000..cbcb80f3 --- /dev/null +++ b/gpu_bdb/bdb_tools/q17_utils.py @@ -0,0 +1,74 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +q17_gmt_offset = -5.0 +# --store_sales date +q17_year = 2001 +q17_month = 12 + +store_sales_cols = [ + "ss_ext_sales_price", + "ss_sold_date_sk", + "ss_store_sk", + "ss_customer_sk", + "ss_promo_sk", + "ss_item_sk", +] +item_cols = ["i_category", "i_item_sk"] +customer_cols = ["c_customer_sk", "c_current_addr_sk"] +store_cols = ["s_gmt_offset", "s_store_sk"] +date_cols = ["d_date_sk", "d_year", "d_moy"] +customer_address_cols = ["ca_address_sk", "ca_gmt_offset"] +promotion_cols = ["p_channel_email", "p_channel_dmail", "p_channel_tv", "p_promo_sk"] + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + customer_df = table_reader.read("customer", relevant_cols=customer_cols) + store_df = table_reader.read("store", relevant_cols=store_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + customer_address_df = table_reader.read( + "customer_address", relevant_cols=customer_address_cols + ) + promotion_df = table_reader.read("promotion", relevant_cols=promotion_cols) + + if c: + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("customer", customer_df, persist=False) + c.create_table("store", store_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("customer_address", customer_address_df, persist=False) + c.create_table("promotion", promotion_df, persist=False) + + return ( + store_sales_df, + item_df, + customer_df, + store_df, + date_dim_df, + customer_address_df, + promotion_df, + ) + diff --git a/gpu_bdb/bdb_tools/q18_utils.py b/gpu_bdb/bdb_tools/q18_utils.py index 998adcfb..c0a9d45a 100644 --- a/gpu_bdb/bdb_tools/q18_utils.py +++ b/gpu_bdb/bdb_tools/q18_utils.py @@ -18,6 +18,53 @@ import cudf from cudf._lib.strings import find_multiple +from bdb_tools.readers import build_reader + +q18_startDate = "2001-05-02" +# --+90days +q18_endDate = "2001-09-02" + +EOL_CHAR = "รจ" + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + + store_sales_cols = [ + "ss_store_sk", + "ss_sold_date_sk", + "ss_net_paid", + ] + date_cols = ["d_date_sk", "d_date"] + store_cols = ["s_store_sk", "s_store_name"] + + store_sales = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_dim = table_reader.read("date_dim", relevant_cols=date_cols) + store = table_reader.read("store", relevant_cols=store_cols) + + ### splitting by row groups for better parallelism + pr_table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + + product_reviews_cols = ["pr_review_date", "pr_review_content", "pr_review_sk"] + product_reviews = pr_table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols, + ) + + if c: + c.create_table("store", store, persist=False) + c.create_table("store_sales", store_sales, persist=False) + c.create_table("date_dim", date_dim, persist=False) + c.create_table("product_reviews", product_reviews, persist=False) + + return store_sales, date_dim, store, product_reviews + + def create_found_reshaped_with_global_pos(found, targets): """Given the dataframe created by mapping find_targets_in_reviews, create a new dataframe in which the nonzero values in each row are exploded diff --git a/gpu_bdb/bdb_tools/q19_utils.py b/gpu_bdb/bdb_tools/q19_utils.py new file mode 100644 index 00000000..105a914e --- /dev/null +++ b/gpu_bdb/bdb_tools/q19_utils.py @@ -0,0 +1,57 @@ + +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +q19_returns_dates_IN = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] + +eol_char = "รจ" + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + date_dim_cols = ["d_week_seq", "d_date_sk", "d_date"] + date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) + store_returns_cols = ["sr_returned_date_sk", "sr_item_sk", "sr_return_quantity"] + store_returns_df = table_reader.read( + "store_returns", relevant_cols=store_returns_cols + ) + web_returns_cols = ["wr_returned_date_sk", "wr_item_sk", "wr_return_quantity"] + web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) + + ### splitting by row groups for better parallelism + pr_table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + + product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] + product_reviews_df = pr_table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols + ) + + if c: + c.create_table('web_returns', web_returns_df, persist=False) + c.create_table('date_dim', date_dim_df, persist=False) + c.create_table('product_reviews', product_reviews_df, persist=False) + c.create_table('store_returns', store_returns_df, persist=False) + + return date_dim_df, store_returns_df, web_returns_df, product_reviews_df + diff --git a/gpu_bdb/bdb_tools/q20_utils.py b/gpu_bdb/bdb_tools/q20_utils.py index 5871f93e..1373d4be 100644 --- a/gpu_bdb/bdb_tools/q20_utils.py +++ b/gpu_bdb/bdb_tools/q20_utils.py @@ -20,11 +20,45 @@ from bdb_tools.utils import train_clustering_model +from bdb_tools.readers import build_reader + # q20 parameters N_CLUSTERS = 8 CLUSTER_ITERATIONS = 20 N_ITER = 5 +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_cols = [ + "ss_customer_sk", + "ss_ticket_number", + "ss_item_sk", + "ss_net_paid", + ] + store_returns_cols = [ + "sr_item_sk", + "sr_customer_sk", + "sr_ticket_number", + "sr_return_amt", + ] + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + store_returns_df = table_reader.read( + "store_returns", relevant_cols=store_returns_cols + ) + + if c: + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("store_returns", store_returns_df, persist=False) + + return store_sales_df, store_returns_df + + def get_clusters(client, ml_input_df, feature_cols): """ Takes the dask client, kmeans_input_df and feature columns. diff --git a/gpu_bdb/bdb_tools/q21_utils.py b/gpu_bdb/bdb_tools/q21_utils.py new file mode 100644 index 00000000..453aea48 --- /dev/null +++ b/gpu_bdb/bdb_tools/q21_utils.py @@ -0,0 +1,69 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +store_sales_cols = [ + "ss_item_sk", + "ss_store_sk", + "ss_customer_sk", + "ss_ticket_number", + "ss_quantity", + "ss_sold_date_sk", +] +date_cols = ["d_date_sk", "d_year", "d_moy"] +websale_cols = ["ws_item_sk", "ws_bill_customer_sk", "ws_quantity", "ws_sold_date_sk"] +sr_cols = [ + "sr_item_sk", + "sr_customer_sk", + "sr_ticket_number", + "sr_return_quantity", + "sr_returned_date_sk", +] +store_cols = ["s_store_name", "s_store_id", "s_store_sk"] +item_cols = ["i_item_id", "i_item_desc", "i_item_sk"] + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) + date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) + web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) + store_returns_df = table_reader.read("store_returns", relevant_cols=sr_cols) + store_table_df = table_reader.read("store", relevant_cols=store_cols) + item_table_df = table_reader.read("item", relevant_cols=item_cols) + + if c: + c.create_table("store_sales", store_sales_df, persist=False) + c.create_table("date_dim", date_dim_df, persist=False) + c.create_table("item", item_table_df, persist=False) + c.create_table("web_sales", web_sales_df, persist=False) + c.create_table("store_returns", store_returns_df, persist=False) + c.create_table("store", store_table_df, persist=False) + + return ( + store_sales_df, + date_dim_df, + web_sales_df, + store_returns_df, + store_table_df, + item_table_df, + ) + diff --git a/gpu_bdb/bdb_tools/q22_utils.py b/gpu_bdb/bdb_tools/q22_utils.py new file mode 100644 index 00000000..b48ed504 --- /dev/null +++ b/gpu_bdb/bdb_tools/q22_utils.py @@ -0,0 +1,56 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader +from bdb_tools.utils import convert_datestring_to_days + +q22_date = "2001-05-08" +q22_i_current_price_min = "0.98" +q22_i_current_price_max = "1.5" + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + inv_columns = [ + "inv_item_sk", + "inv_warehouse_sk", + "inv_date_sk", + "inv_quantity_on_hand", + ] + inventory = table_reader.read("inventory", relevant_cols=inv_columns) + + item_columns = ["i_item_id", "i_current_price", "i_item_sk"] + item = table_reader.read("item", relevant_cols=item_columns) + + warehouse_columns = ["w_warehouse_sk", "w_warehouse_name"] + warehouse = table_reader.read("warehouse", relevant_cols=warehouse_columns) + + dd_columns = ["d_date_sk", "d_date"] + date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) + date_dim = date_dim.map_partitions(convert_datestring_to_days) + + if c: + c.create_table('inventory', inventory, persist=False) + c.create_table('item', item, persist=False) + c.create_table('warehouse', warehouse, persist=False) + c.create_table('date_dim', date_dim, persist=False) + + return inventory, item, warehouse, date_dim + diff --git a/gpu_bdb/bdb_tools/q23_utils.py b/gpu_bdb/bdb_tools/q23_utils.py new file mode 100644 index 00000000..43ef19a2 --- /dev/null +++ b/gpu_bdb/bdb_tools/q23_utils.py @@ -0,0 +1,45 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +q23_year = 2001 +q23_month = 1 +q23_coefficient = 1.3 + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + + date_cols = ["d_date_sk", "d_year", "d_moy"] + date_df = table_reader.read("date_dim", relevant_cols=date_cols) + + inv_cols = [ + "inv_warehouse_sk", + "inv_item_sk", + "inv_date_sk", + "inv_quantity_on_hand", + ] + inv_df = table_reader.read("inventory", relevant_cols=inv_cols) + + if c: + c.create_table('inventory', inv_df, persist=False) + c.create_table('date_dim', date_df, persist=False) + + return date_df, inv_df + diff --git a/gpu_bdb/bdb_tools/q24_utils.py b/gpu_bdb/bdb_tools/q24_utils.py new file mode 100644 index 00000000..a413eb97 --- /dev/null +++ b/gpu_bdb/bdb_tools/q24_utils.py @@ -0,0 +1,49 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +ws_cols = ["ws_item_sk", "ws_sold_date_sk", "ws_quantity"] +item_cols = ["i_item_sk", "i_current_price"] +imp_cols = [ + "imp_item_sk", + "imp_competitor_price", + "imp_start_date", + "imp_end_date", + "imp_sk", +] +ss_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_quantity"] + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + ### read tables + ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) + item_df = table_reader.read("item", relevant_cols=item_cols) + imp_df = table_reader.read("item_marketprices", relevant_cols=imp_cols) + ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) + + if c: + c.create_table("web_sales", ws_df, persist=False) + c.create_table("item", item_df, persist=False) + c.create_table("item_marketprices", imp_df, persist=False) + c.create_table("store_sales", ss_df, persist=False) + + return ws_df, item_df, imp_df, ss_df + diff --git a/gpu_bdb/bdb_tools/q25_utils.py b/gpu_bdb/bdb_tools/q25_utils.py new file mode 100644 index 00000000..523598f5 --- /dev/null +++ b/gpu_bdb/bdb_tools/q25_utils.py @@ -0,0 +1,80 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import dask_cudf + +from bdb_tools.utils import train_clustering_model + +from bdb_tools.readers import build_reader + +from dask import delayed + +q25_date = "2002-01-02" + +N_CLUSTERS = 8 +CLUSTER_ITERATIONS = 20 +N_ITER = 5 + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ss_cols = ["ss_customer_sk", "ss_sold_date_sk", "ss_ticket_number", "ss_net_paid"] + ws_cols = [ + "ws_bill_customer_sk", + "ws_sold_date_sk", + "ws_order_number", + "ws_net_paid", + ] + datedim_cols = ["d_date_sk", "d_date"] + + ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) + ws_ddf = table_reader.read("web_sales", relevant_cols=ws_cols, index=False) + datedim_ddf = table_reader.read("date_dim", relevant_cols=datedim_cols, index=False) + + if c: + c.create_table("web_sales", ws_ddf, persist=False) + c.create_table("store_sales", ss_ddf, persist=False) + c.create_table("date_dim", datedim_ddf, persist=False) + + return ss_ddf, ws_ddf, datedim_ddf + + +def get_clusters(client, ml_input_df): + + ml_tasks = [ + delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) + for df in ml_input_df.to_delayed() + ] + results_dict = client.compute(*ml_tasks, sync=True) + + output = ml_input_df.index.to_frame().reset_index(drop=True) + + labels_final = dask_cudf.from_cudf( + results_dict["cid_labels"], npartitions=output.npartitions + ) + output["label"] = labels_final.reset_index()[0] + + # Sort based on CDH6.1 q25-result formatting + output = output.sort_values(["cid"]) + + results_dict["cid_labels"] = output + return results_dict + + diff --git a/gpu_bdb/bdb_tools/q26_utils.py b/gpu_bdb/bdb_tools/q26_utils.py new file mode 100644 index 00000000..5f299565 --- /dev/null +++ b/gpu_bdb/bdb_tools/q26_utils.py @@ -0,0 +1,42 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from bdb_tools.readers import build_reader + +Q26_CATEGORY = "Books" +Q26_ITEM_COUNT = 5 +N_CLUSTERS = 8 +CLUSTER_ITERATIONS = 20 +N_ITER = 5 + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + ss_cols = ["ss_customer_sk", "ss_item_sk"] + items_cols = ["i_item_sk", "i_category", "i_class_id"] + + ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) + items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) + + if c: + c.create_table("store_sales", ss_ddf, persist=False) + c.create_table("item", items_ddf, persist=False) + + return ss_ddf, items_ddf + diff --git a/gpu_bdb/bdb_tools/q27_utils.py b/gpu_bdb/bdb_tools/q27_utils.py index 499135b1..167cd2a0 100644 --- a/gpu_bdb/bdb_tools/q27_utils.py +++ b/gpu_bdb/bdb_tools/q27_utils.py @@ -16,6 +16,29 @@ import spacy +from bdb_tools.readers import build_reader + +q27_pr_item_sk = 10002 +EOL_CHAR = "." + +def read_tables(config, c=None): + ### splitting by row groups for better parallelism + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] + product_reviews_df = table_reader.read( + "product_reviews", relevant_cols=product_reviews_cols + ) + + if c: + c.create_table("product_reviews", product_reviews_df, persist=False) + + return product_reviews_df + + def ner_parser(df, col_string, batch_size=256): spacy.require_gpu() nlp = spacy.load("en_core_web_sm") diff --git a/gpu_bdb/bdb_tools/q28_utils.py b/gpu_bdb/bdb_tools/q28_utils.py index 7d3d15ac..140aef6a 100644 --- a/gpu_bdb/bdb_tools/q28_utils.py +++ b/gpu_bdb/bdb_tools/q28_utils.py @@ -33,6 +33,7 @@ from uuid import uuid1 +from bdb_tools.readers import build_reader N_FEATURES = 2 ** 23 # Spark is doing 2^20 ngram_range = (1, 2) @@ -40,6 +41,26 @@ norm = None alternate_sign = False +def read_tables(config, c=None): + ### splitting by row groups for better parallelism + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=True, + ) + + columns = [ + "pr_review_content", + "pr_review_rating", + "pr_review_sk", + ] + pr_df = table_reader.read("product_reviews", relevant_cols=columns) + + if c: + c.create_table("product_reviews", pr_df, persist=False) + + return pr_df + def gpu_hashing_vectorizer(x): vec = HashingVectorizer(n_features=N_FEATURES, diff --git a/gpu_bdb/bdb_tools/q29_utils.py b/gpu_bdb/bdb_tools/q29_utils.py new file mode 100644 index 00000000..b0e0cd8f --- /dev/null +++ b/gpu_bdb/bdb_tools/q29_utils.py @@ -0,0 +1,38 @@ + +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +q29_limit = 100 + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], basepath=config["data_dir"], + ) + item_cols = ["i_item_sk", "i_category_id"] + item_df = table_reader.read("item", relevant_cols=item_cols) + + ws_cols = ["ws_order_number", "ws_item_sk"] + ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) + + if c: + c.create_table('item', item_df, persist=False) + c.create_table('web_sales', ws_df, persist=False) + + return item_df, ws_df + diff --git a/gpu_bdb/bdb_tools/q30_utils.py b/gpu_bdb/bdb_tools/q30_utils.py new file mode 100644 index 00000000..2d8e3309 --- /dev/null +++ b/gpu_bdb/bdb_tools/q30_utils.py @@ -0,0 +1,44 @@ +# +# Copyright (c) 2019-2022, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from bdb_tools.readers import build_reader + +# session timeout in secs +q30_session_timeout_inSec = 3600 +# query output limit +q30_limit = 40 + + + +def read_tables(config, c=None): + table_reader = build_reader( + data_format=config["file_format"], + basepath=config["data_dir"], + split_row_groups=config["split_row_groups"], + ) + + item_cols = ["i_category_id", "i_item_sk"] + item_df = table_reader.read("item", relevant_cols=item_cols) + + wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] + wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) + + if c: + c.create_table('web_clickstreams', wcs_df, persist=False) + c.create_table('item', item_df, persist=False) + + return item_df + diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01.py b/gpu_bdb/queries/q01/gpu_bdb_query_01.py index 041f674c..912af07d 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01.py @@ -17,6 +17,11 @@ from bdb_tools.utils import benchmark, gpubdb_argparser, run_query from bdb_tools.readers import build_reader +from bdb_tools.q01_utils import ( + q01_viewed_together_count, + q01_limit, + read_tables +) ### Implementation Notes: # `drop_duplicates` and `groupby` by default brings result to single partition @@ -26,30 +31,8 @@ ### Future Notes: # Settinng index + merge using map_parition can be a work-around if dask native merge is slow - -# -------- Q1 ----------- -q01_i_category_id_IN = [1, 2, 3] -# -- sf1 -> 11 stores, 90k sales in 820k lines -q01_ss_store_sk_IN = [10, 20, 33, 40, 50] -q01_viewed_together_count = 50 -q01_limit = 100 - - -item_cols = ["i_item_sk", "i_category_id"] -ss_cols = ["ss_item_sk", "ss_store_sk", "ss_ticket_number"] - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_df = table_reader.read("item", relevant_cols=item_cols) - ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - return item_df, ss_df - +q01_i_category_id_IN = ["1", "2"," 3"] +q01_ss_store_sk_IN = ["10", "20", "33", "40", "50"] ### Inner Self join to get pairs # Select t1.ss_item_sk as item_sk_1 , t2.ss_item_sk as item_sk_2 diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index f025df86..504497ea 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -26,36 +26,19 @@ ) from bdb_tools.readers import build_reader -from dask.distributed import wait +from bdb_tools.q01_utils import ( + q01_viewed_together_count, + q01_limit, + read_tables +) +from dask.distributed import wait -# -------- Q1 ----------- q01_i_category_id_IN = "1, 2, 3" -# -- sf1 -> 11 stores, 90k sales in 820k lines -q01_ss_store_sk_IN = "10, 20, 33, 40, 50" -q01_viewed_together_count = 50 -q01_limit = 100 - - -item_cols = ["i_item_sk", "i_category_id"] -ss_cols = ["ss_item_sk", "ss_store_sk", "ss_ticket_number"] - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_df = table_reader.read("item", relevant_cols=item_cols) - ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - - c.create_table("item", item_df, persist=False) - c.create_table("store_sales", ss_df, persist=False) +q01_ss_store_sk_IN = ["10", "20", "33", "40", "50"] def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_distinct = f""" SELECT DISTINCT ss_item_sk, ss_ticket_number diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02.py b/gpu_bdb/queries/q02/gpu_bdb_query_02.py index cc7cb5a5..8b6ac390 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02.py @@ -21,6 +21,13 @@ ) from bdb_tools.readers import build_reader from bdb_tools.sessionization import get_distinct_sessions +from bdb_tools.q02_utils import ( + q02_item_sk, + q02_MAX_ITEMS_PER_BASKET, + q02_limit, + q02_session_timeout_inSec, + read_tables +) ### Implementation Notes: @@ -28,13 +35,6 @@ # The bottleneck of current implimenation is `set-index`, once ucx is working correctly # it should go away -# -------- Q2 ----------- -q02_item_sk = 10001 -q02_MAX_ITEMS_PER_BASKET = 5000000 -q02_limit = 30 -q02_session_timeout_inSec = 3600 - - def get_relevant_item_series(df, q02_item_sk): """ Returns relevant items directly @@ -65,13 +65,6 @@ def reduction_function(df, q02_session_timeout_inSec): return grouped_df -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) return wcs_df diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index ecee340a..ef82a263 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -26,30 +26,21 @@ ) from bdb_tools.readers import build_reader -from bdb_tools.sessionization import get_distinct_sessions -from dask.distributed import wait - - -# -------- Q2 ----------- -q02_item_sk = 10001 -q02_limit = 30 -q02_session_timeout_inSec = 3600 +from bdb_tools.sessionization import get_distinct_sessions -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] - wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) +from bdb_tools.q02_utils import ( + q02_item_sk, + q02_limit, + q02_session_timeout_inSec, + read_tables +) - c.create_table("web_clickstreams", wcs_df, persist=False) +from dask.distributed import wait def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = """ SELECT diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03.py b/gpu_bdb/queries/q03/gpu_bdb_query_03.py index 073d3374..2404e357 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03.py @@ -24,23 +24,22 @@ from bdb_tools.readers import build_reader from bdb_tools.q03_utils import ( - apply_find_items_viewed + apply_find_items_viewed, + q03_days_in_sec_before_purchase, + q03_views_before_purchase, + q03_purchased_item_IN, + q03_limit, + read_tables ) from distributed import wait import numpy as np -from numba import cuda import glob from dask import delayed -q03_days_in_sec_before_purchase = 864000 -q03_views_before_purchase = 5 -q03_purchased_item_IN = 10001 q03_purchased_item_category_IN = [2, 3] -q03_limit = 100 - def get_wcs_minima(config): import dask_cudf @@ -110,18 +109,6 @@ def reduction_function(df, item_df_filtered): return grouped_df -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_cols = ["i_category_id", "i_item_sk"] - item_df = table_reader.read("item", relevant_cols=item_cols) - return item_df - - def main(client, config): import dask_cudf import cudf diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index e3dc3a86..3a0cbe33 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -25,46 +25,21 @@ from bdb_tools.readers import build_reader from bdb_tools.q03_utils import ( - apply_find_items_viewed + apply_find_items_viewed, + q03_days_in_sec_before_purchase, + q03_views_before_purchase, + q03_purchased_item_IN, + q03_limit, + read_tables ) from dask.distributed import wait -# -------- Q03 ----------- -q03_days_in_sec_before_purchase = 864000 -q03_views_before_purchase = 5 -q03_purchased_item_IN = 10001 -# --see q1 for categories q03_purchased_item_category_IN = "2,3" -q03_limit = 100 - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_cols = ["i_category_id", "i_item_sk"] - wcs_cols = [ - "wcs_user_sk", - "wcs_click_time_sk", - "wcs_click_date_sk", - "wcs_item_sk", - "wcs_sales_sk", - ] - - item_df = table_reader.read("item", relevant_cols=item_cols) - wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - - c.create_table("web_clickstreams", wcs_df, persist=False) - c.create_table("item", item_df, persist=False) - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = """ SELECT i_item_sk, diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04.py b/gpu_bdb/queries/q04/gpu_bdb_query_04.py index 9300ae70..9134f99c 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04.py @@ -28,35 +28,13 @@ from bdb_tools.q04_utils import ( abandonedShoppingCarts, - reduction_function + reduction_function, + read_tables ) # parameters q04_session_timeout_inSec = 3600 - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - wp_cols = ["wp_type", "wp_web_page_sk"] - wp_df = table_reader.read("web_page", relevant_cols=wp_cols) - - wcs_cols = [ - "wcs_user_sk", - "wcs_click_date_sk", - "wcs_click_time_sk", - "wcs_web_page_sk", - "wcs_sales_sk", - ] - web_clicksteams_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - - return wp_df, web_clicksteams_df - - def main(client, config): import cudf diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index ad33cc0e..e37468b4 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -31,37 +31,15 @@ from bdb_tools.q04_utils import ( abandonedShoppingCarts, - reduction_function + reduction_function, + read_tables ) from dask.distributed import wait -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - wp_cols = ["wp_type", "wp_web_page_sk"] - wp_df = table_reader.read("web_page", relevant_cols=wp_cols) - - wcs_cols = [ - "wcs_user_sk", - "wcs_click_date_sk", - "wcs_click_time_sk", - "wcs_web_page_sk", - "wcs_sales_sk", - ] - wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - - c.create_table('web_page_wo_categorical', wp_df, persist=False) - c.create_table('web_clickstreams', wcs_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_web_page = """ SELECT wp_type, wp_web_page_sk diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05.py b/gpu_bdb/queries/q05/gpu_bdb_query_05.py index 3d96122b..f1e3a4f0 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05.py @@ -24,7 +24,10 @@ ) from bdb_tools.readers import build_reader -from bdb_tools.q05_utils import build_and_predict_model +from bdb_tools.q05_utils import ( + build_and_predict_model, + read_tables +) import cupy as cp import numpy as np @@ -39,28 +42,6 @@ Q05_I_CATEGORY = "Books" wcs_columns = ["wcs_item_sk", "wcs_user_sk"] -items_columns = ["i_item_sk", "i_category", "i_category_id"] -customer_columns = ["c_customer_sk", "c_current_cdemo_sk"] -customer_dem_columns = ["cd_demo_sk", "cd_gender", "cd_education_status"] - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_ddf = table_reader.read("item", relevant_cols=items_columns, index=False) - customer_ddf = table_reader.read( - "customer", relevant_cols=customer_columns, index=False - ) - customer_dem_ddf = table_reader.read( - "customer_demographics", relevant_cols=customer_dem_columns, index=False - ) - - return (item_ddf, customer_ddf, customer_dem_ddf) - def get_groupby_results(file_list, item_df): """ diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index 3eb4dc4a..6ad5ef1c 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -24,40 +24,13 @@ run_query, ) from bdb_tools.readers import build_reader -from bdb_tools.q05_utils import build_and_predict_model - - -wcs_columns = ["wcs_item_sk", "wcs_user_sk"] -items_columns = ["i_item_sk", "i_category", "i_category_id"] -customer_columns = ["c_customer_sk", "c_current_cdemo_sk"] -customer_dem_columns = ["cd_demo_sk", "cd_gender", "cd_education_status"] - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_ddf = table_reader.read("item", relevant_cols=items_columns, index=False) - customer_ddf = table_reader.read( - "customer", relevant_cols=customer_columns, index=False - ) - customer_dem_ddf = table_reader.read( - "customer_demographics", relevant_cols=customer_dem_columns, index=False - ) - wcs_ddf = table_reader.read( - "web_clickstreams", relevant_cols=wcs_columns, index=False - ) - - c.create_table("web_clickstreams", wcs_ddf, persist=False) - c.create_table("customer", customer_ddf, persist=False) - c.create_table("item", item_ddf, persist=False) - c.create_table("customer_demographics", customer_dem_ddf, persist=False) - +from bdb_tools.q05_utils import ( + build_and_predict_model, + read_tables +) def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = """ SELECT diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06.py b/gpu_bdb/queries/q06/gpu_bdb_query_06.py index b7326ab3..d621cf2e 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06.py @@ -25,53 +25,11 @@ from bdb_tools.readers import build_reader from distributed import wait - -q06_YEAR = 2001 -q6_limit_rows = 100 - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - web_sales_cols = [ - "ws_bill_customer_sk", - "ws_sold_date_sk", - "ws_ext_list_price", - "ws_ext_wholesale_cost", - "ws_ext_discount_amt", - "ws_ext_sales_price", - ] - store_sales_cols = [ - "ss_customer_sk", - "ss_sold_date_sk", - "ss_ext_list_price", - "ss_ext_wholesale_cost", - "ss_ext_discount_amt", - "ss_ext_sales_price", - ] - date_cols = ["d_date_sk", "d_year", "d_moy"] - customer_cols = [ - "c_customer_sk", - "c_customer_id", - "c_email_address", - "c_first_name", - "c_last_name", - "c_preferred_cust_flag", - "c_birth_country", - "c_login", - ] - - ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) - ss_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_df = table_reader.read("date_dim", relevant_cols=date_cols) - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - - return (ws_df, ss_df, date_df, customer_df) - +from bdb_tools.q06_utils import ( + q06_YEAR, + q06_LIMIT, + read_tables +) def get_sales_ratio(df, table="store_sales"): assert table in ("store_sales", "web_sales") @@ -247,7 +205,7 @@ def main(client, config): ) ) - return result_df.head(q6_limit_rows) + return result_df.head(q06_LIMIT) if __name__ == "__main__": diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index 2ab18d2c..a5ea5328 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -27,61 +27,14 @@ from bdb_tools.readers import build_reader - -# -------- Q6 ----------- -q06_LIMIT = 100 -# --web_sales and store_sales date -q06_YEAR = 2001 - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - web_sales_cols = [ - "ws_bill_customer_sk", - "ws_sold_date_sk", - "ws_ext_list_price", - "ws_ext_wholesale_cost", - "ws_ext_discount_amt", - "ws_ext_sales_price", - ] - store_sales_cols = [ - "ss_customer_sk", - "ss_sold_date_sk", - "ss_ext_list_price", - "ss_ext_wholesale_cost", - "ss_ext_discount_amt", - "ss_ext_sales_price", - ] - date_cols = ["d_date_sk", "d_year", "d_moy"] - customer_cols = [ - "c_customer_sk", - "c_customer_id", - "c_email_address", - "c_first_name", - "c_last_name", - "c_preferred_cust_flag", - "c_birth_country", - "c_login", - ] - - ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) - ss_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_df = table_reader.read("date_dim", relevant_cols=date_cols) - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - - c.create_table('web_sales', ws_df, persist=False) - c.create_table('store_sales', ss_df, persist=False) - c.create_table('date_dim', date_df, persist=False) - c.create_table('customer', customer_df, persist=False) - +from bdb_tools.q06_utils import ( + q06_LIMIT, + q06_YEAR, + read_tables +) def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = f""" WITH temp_table_1 as diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07.py b/gpu_bdb/queries/q07/gpu_bdb_query_07.py index a14cbcfd..6f0350c5 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07.py @@ -24,6 +24,7 @@ ) from bdb_tools.readers import build_reader +from bdb_tools.q07_utils import read_tables q07_HIGHER_PRICE_RATIO = 1.2 # --store_sales date @@ -51,44 +52,10 @@ def create_high_price_items_df(item_df): return high_price_items_df -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_cols = ["i_item_sk", "i_current_price", "i_category"] - store_sales_cols = ["ss_item_sk", "ss_customer_sk", "ss_sold_date_sk"] - store_cols = ["s_store_sk"] - date_cols = ["d_date_sk", "d_year", "d_moy"] - customer_cols = ["c_customer_sk", "c_current_addr_sk"] - customer_address_cols = ["ca_address_sk", "ca_state"] - - item_df = table_reader.read("item", relevant_cols=item_cols) - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - store_df = table_reader.read("store", relevant_cols=store_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - customer_address_df = table_reader.read( - "customer_address", relevant_cols=customer_address_cols - ) - - return ( - item_df, - store_sales_df, - store_df, - date_dim_df, - customer_df, - customer_address_df, - ) - - def main(client, config): ( item_df, store_sales_df, - store_df, date_dim_df, customer_df, customer_address_df, diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index 58fcdbf1..3133500c 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -28,39 +28,10 @@ from bdb_tools.readers import build_reader - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_cols = ["i_item_sk", "i_current_price", "i_category"] - store_sales_cols = ["ss_item_sk", "ss_customer_sk", "ss_sold_date_sk"] - store_cols = ["s_store_sk"] - date_cols = ["d_date_sk", "d_year", "d_moy"] - customer_cols = ["c_customer_sk", "c_current_addr_sk"] - customer_address_cols = ["ca_address_sk", "ca_state"] - - item_df = table_reader.read("item", relevant_cols=item_cols) - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - # store_df = table_reader.read("store", relevant_cols=store_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - customer_address_df = table_reader.read( - "customer_address", relevant_cols=customer_address_cols - ) - - c.create_table("item", item_df, persist=False) - c.create_table("customer", customer_df, persist=False) - c.create_table("store_sales", store_sales_df, persist=False) - c.create_table("date_dim", date_dim_df, persist=False) - c.create_table("customer_address", customer_address_df, persist=False) - +from bdb_tools.q07_utils import read_tables def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = """ WITH temp_table as diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08.py b/gpu_bdb/queries/q08/gpu_bdb_query_08.py index cc7b725e..f0c59fb5 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08.py @@ -29,17 +29,15 @@ get_sessions, get_unique_sales_keys_from_sessions, prep_for_sessionization, + q08_STARTDATE, + q08_ENDDATE, + read_tables ) import numpy as np from distributed import wait from dask import delayed - -q08_STARTDATE = "2001-09-02" -q08_ENDDATE = "2002-09-02" - - def etl_wcs(wcs_fn, filtered_date_df, web_page_df): import cudf @@ -81,24 +79,6 @@ def etl_wcs(wcs_fn, filtered_date_df, web_page_df): return merged_df[cols_to_keep] -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - date_dim_cols = ["d_date_sk", "d_date"] - web_page_cols = ["wp_web_page_sk", "wp_type"] - web_sales_cols = ["ws_net_paid", "ws_order_number", "ws_sold_date_sk"] - - date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) - web_page_df = table_reader.read("web_page", relevant_cols=web_page_cols) - web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) - - return (date_dim_df, web_page_df, web_sales_df) - - def reduction_function(df, REVIEW_CAT_CODE): # category code of review records diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index f15d8229..38f8b086 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -30,47 +30,15 @@ get_sessions, get_unique_sales_keys_from_sessions, prep_for_sessionization, + q08_STARTDATE, + q08_ENDDATE, + read_tables ) from dask.distributed import wait - -# -------- Q8 ----------- -q08_STARTDATE = "2001-09-02" -q08_ENDDATE = "2002-09-02" - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - date_dim_cols = ["d_date_sk", "d_date"] - web_page_cols = ["wp_web_page_sk", "wp_type"] - web_sales_cols = ["ws_net_paid", "ws_order_number", "ws_sold_date_sk"] - wcs_cols = [ - "wcs_user_sk", - "wcs_sales_sk", - "wcs_click_date_sk", - "wcs_click_time_sk", - "wcs_web_page_sk", - ] - - date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) - web_page_df = table_reader.read("web_page", relevant_cols=web_page_cols) - web_sales_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) - wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - - c.create_table("web_clickstreams", wcs_df, persist=False) - c.create_table("web_sales", web_sales_df, persist=False) - c.create_table("web_page", web_page_df, persist=False) - c.create_table("date_dim", date_dim_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = f""" SELECT d_date_sk diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09.py b/gpu_bdb/queries/q09/gpu_bdb_query_09.py index 8c4bc9d8..60ea0c2b 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09.py @@ -24,76 +24,38 @@ ) from bdb_tools.readers import build_reader - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ss_columns = [ - "ss_quantity", - "ss_sold_date_sk", - "ss_addr_sk", - "ss_store_sk", - "ss_cdemo_sk", - "ss_sales_price", - "ss_net_profit", - ] - - store_sales = table_reader.read("store_sales", relevant_cols=ss_columns) - - ca_columns = ["ca_address_sk", "ca_country", "ca_state"] - customer_address = table_reader.read("customer_address", relevant_cols=ca_columns) - - cd_columns = ["cd_demo_sk", "cd_marital_status", "cd_education_status"] - customer_demographics = table_reader.read( - "customer_demographics", relevant_cols=cd_columns - ) - - dd_columns = ["d_year", "d_date_sk"] - date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) - - s_columns = ["s_store_sk"] - store = table_reader.read("store", relevant_cols=s_columns) - - return store_sales, customer_address, customer_demographics, date_dim, store - +from bdb_tools.q09_utils import ( + q09_year, + q09_part1_ca_country, + q09_part1_net_profit_min, + q09_part1_net_profit_max, + q09_part1_education_status, + q09_part1_marital_status, + q09_part1_sales_price_min, + q09_part1_sales_price_max, + q09_part2_ca_country, + q09_part2_net_profit_min, + q09_part2_net_profit_max, + q09_part2_education_status, + q09_part2_marital_status, + q09_part2_sales_price_min, + q09_part2_sales_price_max, + q09_part3_ca_country, + q09_part3_net_profit_min, + q09_part3_net_profit_max, + q09_part3_education_status, + q09_part3_marital_status, + q09_part3_sales_price_min, + q09_part3_sales_price_max, + read_tables +) def main(client, config): import cudf - # Conf variables - - q09_year = 2001 - - q09_part1_ca_country = "United States" q09_part1_ca_state_IN = "KY", "GA", "NM" - q09_part1_net_profit_min = 0 - q09_part1_net_profit_max = 2000 - q09_part1_education_status = "4 yr Degree" - q09_part1_marital_status = "M" - q09_part1_sales_price_min = 100 - q09_part1_sales_price_max = 150 - - q09_part2_ca_country = "United States" q09_part2_ca_state_IN = "MT", "OR", "IN" - q09_part2_net_profit_min = 150 - q09_part2_net_profit_max = 3000 - q09_part2_education_status = "4 yr Degree" - q09_part2_marital_status = "M" - q09_part2_sales_price_min = 50 - q09_part2_sales_price_max = 200 - - q09_part3_ca_country = "United States" q09_part3_ca_state_IN = "WI", "MO", "WV" - q09_part3_net_profit_min = 50 - q09_part3_net_profit_max = 25000 - q09_part3_education_status = "4 yr Degree" - q09_part3_marital_status = "M" - q09_part3_sales_price_min = 150 - q09_part3_sales_price_max = 200 ( store_sales, diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index 8c5ef131..c1243ec0 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -27,80 +27,39 @@ from bdb_tools.readers import build_reader +from bdb_tools.q09_utils import ( + q09_year, + q09_part1_ca_country, + q09_part1_net_profit_min, + q09_part1_net_profit_max, + q09_part1_education_status, + q09_part1_marital_status, + q09_part1_sales_price_min, + q09_part1_sales_price_max, + q09_part2_ca_country, + q09_part2_net_profit_min, + q09_part2_net_profit_max, + q09_part2_education_status, + q09_part2_marital_status, + q09_part2_sales_price_min, + q09_part2_sales_price_max, + q09_part3_ca_country, + q09_part3_net_profit_min, + q09_part3_net_profit_max, + q09_part3_education_status, + q09_part3_marital_status, + q09_part3_sales_price_min, + q09_part3_sales_price_max, + read_tables +) -# -------- Q9 ----------- -q09_year = 2001 -q09_part1_ca_country = "United States" q09_part1_ca_state_IN = "'KY', 'GA', 'NM'" -q09_part1_net_profit_min = 0 -q09_part1_net_profit_max = 2000 -q09_part1_education_status = "4 yr Degree" -q09_part1_marital_status = "M" -q09_part1_sales_price_min = 100 -q09_part1_sales_price_max = 150 - -q09_part2_ca_country = "United States" q09_part2_ca_state_IN = "'MT', 'OR', 'IN'" -q09_part2_net_profit_min = 150 -q09_part2_net_profit_max = 3000 -q09_part2_education_status = "4 yr Degree" -q09_part2_marital_status = "M" -q09_part2_sales_price_min = 50 -q09_part2_sales_price_max = 200 - -q09_part3_ca_country = "United States" q09_part3_ca_state_IN = "'WI', 'MO', 'WV'" -q09_part3_net_profit_min = 50 -q09_part3_net_profit_max = 25000 -q09_part3_education_status = "4 yr Degree" -q09_part3_marital_status = "M" -q09_part3_sales_price_min = 150 -q09_part3_sales_price_max = 200 - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ss_columns = [ - "ss_quantity", - "ss_sold_date_sk", - "ss_addr_sk", - "ss_store_sk", - "ss_cdemo_sk", - "ss_sales_price", - "ss_net_profit", - ] - - store_sales = table_reader.read("store_sales", relevant_cols=ss_columns) - - ca_columns = ["ca_address_sk", "ca_country", "ca_state"] - customer_address = table_reader.read("customer_address", relevant_cols=ca_columns) - - cd_columns = ["cd_demo_sk", "cd_marital_status", "cd_education_status"] - customer_demographics = table_reader.read( - "customer_demographics", relevant_cols=cd_columns - ) - - dd_columns = ["d_year", "d_date_sk"] - date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) - - s_columns = ["s_store_sk"] - store = table_reader.read("store", relevant_cols=s_columns) - - c.create_table("store_sales", store_sales, persist=False) - c.create_table("customer_address", customer_address, persist=False) - c.create_table("customer_demographics", customer_demographics, persist=False) - c.create_table("date_dim", date_dim, persist=False) - c.create_table("store", store, persist=False) - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = f""" SELECT SUM(ss1.ss_quantity) diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10.py b/gpu_bdb/queries/q10/gpu_bdb_query_10.py index cb24ef88..2cb59378 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10.py @@ -23,36 +23,18 @@ run_query, ) from bdb_tools.text import create_sentences_from_reviews, create_words_from_sentences - +from bdb_tools.readers import build_reader +from bdb_tools.q10_utils import ( + eol_char, + read_tables +) import rmm import cupy as cp import distributed -from bdb_tools.readers import build_reader from dask.distributed import Client, wait - -# -------- Q10 ----------- -eol_char = "รจ" - - -def read_tables(config): - - ### splitting by row groups for better parallelism - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] - - product_reviews_df = table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols, - ) - return product_reviews_df - - def load_sentiment_words(filename, sentiment): import cudf diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index 5f8cff83..dd8d83c0 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -34,31 +34,15 @@ from bdb_tools.readers import build_reader -from dask.distributed import wait - - -eol_char = "รจ" - - -def read_tables(data_dir, c, config): - - ### splitting by row groups for better parallelism - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] - - product_reviews_df = table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols, - ) - - c.create_table("product_reviews", product_reviews_df, persist=False) +from bdb_tools.q10_utils import ( + eol_char, + read_tables +) +from dask.distributed import wait def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = """ SELECT pr_item_sk, diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11.py b/gpu_bdb/queries/q11/gpu_bdb_query_11.py index 6ff0b5b3..bc0199b1 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11.py @@ -25,42 +25,13 @@ ) from bdb_tools.readers import build_reader -from numba import cuda -import numpy as np +from bdb_tools.q11_utils import read_tables +import numpy as np q11_start_date = "2003-01-02" q11_end_date = "2003-02-02" - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - product_review_cols = [ - "pr_review_rating", - "pr_item_sk", - ] - web_sales_cols = [ - "ws_sold_date_sk", - "ws_net_paid", - "ws_item_sk", - ] - date_cols = ["d_date_sk", "d_date"] - - pr_df = table_reader.read("product_reviews", relevant_cols=product_review_cols) - # we only read int columns here so it should scale up to sf-10k as just 26M rows - pr_df = pr_df.repartition(npartitions=1) - - ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) - date_df = table_reader.read("date_dim", relevant_cols=date_cols) - - return pr_df, ws_df, date_df - - def main(client, config): import cudf diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index 7b49d7f0..9e69cbff 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -28,39 +28,10 @@ from bdb_tools.readers import build_reader - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - product_review_cols = [ - "pr_review_rating", - "pr_item_sk", - ] - web_sales_cols = [ - "ws_sold_date_sk", - "ws_net_paid", - "ws_item_sk", - ] - date_cols = ["d_date_sk", "d_date"] - - pr_df = table_reader.read("product_reviews", relevant_cols=product_review_cols) - # we only read int columns here so it should scale up to sf-10k as just 26M rows - pr_df = pr_df.repartition(npartitions=1) - - ws_df = table_reader.read("web_sales", relevant_cols=web_sales_cols) - date_df = table_reader.read("date_dim", relevant_cols=date_cols) - - c.create_table("web_sales", ws_df, persist=False) - c.create_table("product_reviews", pr_df, persist=False) - c.create_table("date_dim", date_df, persist=False) - +from bdb_tools.q11_utils import read_tables def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = """ WITH p AS diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12.py b/gpu_bdb/queries/q12/gpu_bdb_query_12.py index e912c6f3..8c17e8e2 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12.py @@ -24,6 +24,7 @@ run_query, ) from bdb_tools.readers import build_reader +from bdb_tools.q12_utils import read_tables from distributed import wait import numpy as np @@ -36,17 +37,11 @@ ### These parameters are not used -# q12_startDate='2001-09-02' -# q12_endDate1='2001-10-02' -# q12_endDate2='2001-12-02' q12_i_category_IN = ["Books", "Electronics"] ### below was hard coded in the orignal query q12_store_sale_sk_start_date = 37134 -item_cols = ["i_item_sk", "i_category"] -store_sales_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_customer_sk"] - ### Util Functions def string_filter(df, col_name, col_values): """ @@ -63,19 +58,6 @@ def string_filter(df, col_name, col_values): return df[bool_flag].reset_index(drop=True) -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_df = table_reader.read("item", relevant_cols=item_cols) - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - - return item_df, store_sales_df - - def filter_wcs_table(web_clickstreams_fn, filtered_item_df): """ Filter web clickstreams table diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index c3e93b1c..b55471e3 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -27,33 +27,12 @@ from bdb_tools.readers import build_reader +from bdb_tools.q12_utils import read_tables -# -------- Q12 ----------- q12_i_category_IN = "'Books', 'Electronics'" -item_cols = ["i_item_sk", "i_category"] -store_sales_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_customer_sk"] -wcs_cols = ["wcs_user_sk", "wcs_click_date_sk", "wcs_item_sk", "wcs_sales_sk"] - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_df = table_reader.read("item", relevant_cols=item_cols) - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - - c.create_table("web_clickstreams", wcs_df, persist=False) - c.create_table("store_sales", store_sales_df, persist=False) - c.create_table("item", item_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = f""" SELECT DISTINCT wcs_user_sk diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13.py b/gpu_bdb/queries/q13/gpu_bdb_query_13.py index a61dbfec..316c4a96 100755 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13.py @@ -22,6 +22,8 @@ run_query, ) from bdb_tools.readers import build_reader +from bdb_tools.q13_utils import read_tables + from distributed import wait @@ -46,28 +48,6 @@ def get_sales_ratio(df): return df -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - date_cols = ["d_date_sk", "d_year"] - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - - customer_cols = ["c_customer_sk", "c_customer_id", "c_first_name", "c_last_name"] - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - - s_sales_cols = ["ss_sold_date_sk", "ss_customer_sk", "ss_net_paid"] - s_sales_df = table_reader.read("store_sales", relevant_cols=s_sales_cols) - - w_sales_cols = ["ws_sold_date_sk", "ws_bill_customer_sk", "ws_net_paid"] - web_sales_df = table_reader.read("web_sales", relevant_cols=w_sales_cols) - - return date_dim_df, customer_df, s_sales_df, web_sales_df - - def main(client, config): date_dim_df, customer_df, s_sales_df, web_sales_df = benchmark( read_tables, diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index 2ac68b3e..c9c7da30 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -28,36 +28,12 @@ from bdb_tools.readers import build_reader -from dask.distributed import wait - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - date_cols = ["d_date_sk", "d_year"] - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - - customer_cols = ["c_customer_sk", "c_customer_id", "c_first_name", "c_last_name"] - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - - s_sales_cols = ["ss_sold_date_sk", "ss_customer_sk", "ss_net_paid"] - s_sales_df = table_reader.read("store_sales", relevant_cols=s_sales_cols) - - w_sales_cols = ["ws_sold_date_sk", "ws_bill_customer_sk", "ws_net_paid"] - web_sales_df = table_reader.read("web_sales", relevant_cols=w_sales_cols) - - c.create_table("date_dim", date_dim_df, persist=False) - c.create_table("customer", customer_df, persist=False) - c.create_table("store_sales", s_sales_df, persist=False) - c.create_table("web_sales", web_sales_df, persist=False) +from bdb_tools.q13_utils import read_tables +from dask.distributed import wait def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = """ SELECT diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14.py b/gpu_bdb/queries/q14/gpu_bdb_query_14.py index 52cbc09f..6cb7a605 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14.py @@ -26,31 +26,7 @@ run_query, ) from bdb_tools.readers import build_reader - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ws_columns = ["ws_ship_hdemo_sk", "ws_web_page_sk", "ws_sold_time_sk"] - web_sales = table_reader.read("web_sales", relevant_cols=ws_columns) - - hd_columns = ["hd_demo_sk", "hd_dep_count"] - household_demographics = table_reader.read( - "household_demographics", relevant_cols=hd_columns - ) - - wp_columns = ["wp_web_page_sk", "wp_char_count"] - web_page = table_reader.read("web_page", relevant_cols=wp_columns) - - td_columns = ["t_time_sk", "t_hour"] - time_dim = table_reader.read("time_dim", relevant_cols=td_columns) - - return web_sales, household_demographics, web_page, time_dim - +from bdb_tools.q14_utils import read_tables def main(client, config): import cudf diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index b5cd4873..aaea754f 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -27,36 +27,10 @@ from bdb_tools.readers import build_reader - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ws_columns = ["ws_ship_hdemo_sk", "ws_web_page_sk", "ws_sold_time_sk"] - web_sales = table_reader.read("web_sales", relevant_cols=ws_columns) - - hd_columns = ["hd_demo_sk", "hd_dep_count"] - household_demographics = table_reader.read( - "household_demographics", relevant_cols=hd_columns - ) - - wp_columns = ["wp_web_page_sk", "wp_char_count"] - web_page = table_reader.read("web_page", relevant_cols=wp_columns) - - td_columns = ["t_time_sk", "t_hour"] - time_dim = table_reader.read("time_dim", relevant_cols=td_columns) - - c.create_table("household_demographics", household_demographics, persist=False) - c.create_table("web_page", web_page, persist=False) - c.create_table("web_sales", web_sales, persist=False) - c.create_table("time_dim", time_dim, persist=False) - +from bdb_tools.q14_utils import read_tables def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = """ SELECT CASE WHEN pmc > 0.0 THEN CAST (amc AS DOUBLE) / CAST (pmc AS DOUBLE) ELSE -1.0 END AS am_pm_ratio diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15.py b/gpu_bdb/queries/q15/gpu_bdb_query_15.py index 7699d087..7249051f 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15.py @@ -25,34 +25,17 @@ convert_datestring_to_days, ) from bdb_tools.readers import build_reader +from bdb_tools.q15_utils import ( + q15_startDate, + q15_endDate, + q15_store_sk, + store_sales_cols, + read_tables +) import datetime import numpy as np - -q15_startDate = "2001-09-02" -q15_endDate = "2002-09-02" -q15_store_sk = "10" - -store_sales_cols = ["ss_sold_date_sk", "ss_net_paid", "ss_store_sk", "ss_item_sk"] -date_cols = ["d_date", "d_date_sk"] -item_cols = ["i_item_sk", "i_category_id"] - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - - return store_sales_df, date_dim_df, item_df - - def main(client, config): store_sales_df, date_dim_df, item_df = benchmark( diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index 3c06393d..84783fd9 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -27,37 +27,15 @@ from bdb_tools.readers import build_reader - -# -------- Q15 ----------- -# --store_sales date range -q15_startDate = "2001-09-02" -# --+1year -q15_endDate = "2002-09-02" -q15_store_sk = 10 - -store_sales_cols = ["ss_sold_date_sk", "ss_net_paid", "ss_store_sk", "ss_item_sk"] -date_cols = ["d_date", "d_date_sk"] -item_cols = ["i_item_sk", "i_category_id"] - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - - c.create_table("store_sales", store_sales_df, persist=False) - c.create_table("date_dim", date_dim_df, persist=False) - c.create_table("item", item_df, persist=False) - +from bdb_tools.q15_utils import ( + q15_startDate, + q15_endDate, + q15_store_sk, + read_tables +) def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = f""" SELECT * diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16.py b/gpu_bdb/queries/q16/gpu_bdb_query_16.py index e093427d..be4bd598 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16.py @@ -24,7 +24,8 @@ convert_datestring_to_days, ) from bdb_tools.merge_util import hash_merge -from bdb_tools.readers import build_reader +from bdb_tools.q16_utils import read_tables + from dask.distributed import wait import numpy as np @@ -33,19 +34,6 @@ ### conf q16_date = "2001-03-16" -websale_cols = [ - "ws_order_number", - "ws_item_sk", - "ws_warehouse_sk", - "ws_sold_date_sk", - "ws_sales_price", -] -web_returns_cols = ["wr_order_number", "wr_item_sk", "wr_refunded_cash"] -date_cols = ["d_date", "d_date_sk"] -item_cols = ["i_item_sk", "i_item_id"] -warehouse_cols = ["w_warehouse_sk", "w_state"] - - # INSERT INTO TABLE ${hiveconf:RESULT_TABLE} # SELECT w_state, i_item_id, # SUM( @@ -72,21 +60,6 @@ def get_before_after_sales(df, q16_timestamp): return df -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) - web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - warehouse_df = table_reader.read("warehouse", relevant_cols=warehouse_cols) - return web_sales_df, web_returns_df, date_dim_df, item_df, warehouse_df - - def main(client, config): import cudf diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 2964bdd6..38618dc2 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -29,41 +29,10 @@ from bdb_tools.readers import build_reader - -websale_cols = [ - "ws_order_number", - "ws_item_sk", - "ws_warehouse_sk", - "ws_sold_date_sk", - "ws_sales_price", -] -web_returns_cols = ["wr_order_number", "wr_item_sk", "wr_refunded_cash"] -date_cols = ["d_date", "d_date_sk"] -item_cols = ["i_item_sk", "i_item_id"] -warehouse_cols = ["w_warehouse_sk", "w_state"] - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) - web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - warehouse_df = table_reader.read("warehouse", relevant_cols=warehouse_cols) - - c.create_table("web_sales", web_sales_df, persist=False) - c.create_table("web_returns", web_returns_df, persist=False) - c.create_table("date_dim", date_dim_df, persist=False) - c.create_table("item", item_df, persist=False) - c.create_table("warehouse", warehouse_df, persist=False) - +from bdb_tools.q16_utils import read_tables def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) date = datetime.datetime(2001, 3, 16) start = (date + timedelta(days=-30)).strftime("%Y-%m-%d") diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17.py b/gpu_bdb/queries/q17/gpu_bdb_query_17.py index 8f36a11e..1bd4131b 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17.py @@ -23,61 +23,16 @@ left_semi_join, run_query, ) -from bdb_tools.readers import build_reader - +from bdb_tools.q17_utils import ( + q17_gmt_offset, + q17_year, + q17_month, + store_sales_cols, + read_tables +) -### conf -q17_gmt_offset = -5 -# --store_sales date -q17_year = 2001 -q17_month = 12 q17_i_category_IN = "Books", "Music" - -store_sales_cols = [ - "ss_ext_sales_price", - "ss_sold_date_sk", - "ss_store_sk", - "ss_customer_sk", - "ss_promo_sk", - "ss_item_sk", -] -item_cols = ["i_category", "i_item_sk"] -customer_cols = ["c_customer_sk", "c_current_addr_sk"] -store_cols = ["s_gmt_offset", "s_store_sk"] -date_cols = ["d_date_sk", "d_year", "d_moy"] -customer_address_cols = ["ca_address_sk", "ca_gmt_offset"] -promotion_cols = ["p_channel_email", "p_channel_dmail", "p_channel_tv", "p_promo_sk"] - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - store_df = table_reader.read("store", relevant_cols=store_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - customer_address_df = table_reader.read( - "customer_address", relevant_cols=customer_address_cols - ) - promotion_df = table_reader.read("promotion", relevant_cols=promotion_cols) - - return ( - store_sales_df, - item_df, - customer_df, - store_df, - date_dim_df, - customer_address_df, - promotion_df, - ) - - def main(client, config): import cudf diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 9c20982d..58d44ba1 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -27,57 +27,17 @@ from bdb_tools.readers import build_reader +from bdb_tools.q17_utils import ( + q17_gmt_offset, + q17_year, + q17_month, + read_tables +) -# ------- Q17 ------ -q17_gmt_offset = -5.0 -# --store_sales date -q17_year = 2001 -q17_month = 12 q17_i_category_IN = "'Books', 'Music'" -store_sales_cols = [ - "ss_ext_sales_price", - "ss_sold_date_sk", - "ss_store_sk", - "ss_customer_sk", - "ss_promo_sk", - "ss_item_sk", -] -item_cols = ["i_category", "i_item_sk"] -customer_cols = ["c_customer_sk", "c_current_addr_sk"] -store_cols = ["s_gmt_offset", "s_store_sk"] -date_cols = ["d_date_sk", "d_year", "d_moy"] -customer_address_cols = ["ca_address_sk", "ca_gmt_offset"] -promotion_cols = ["p_channel_email", "p_channel_dmail", "p_channel_tv", "p_promo_sk"] - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - customer_df = table_reader.read("customer", relevant_cols=customer_cols) - store_df = table_reader.read("store", relevant_cols=store_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - customer_address_df = table_reader.read( - "customer_address", relevant_cols=customer_address_cols - ) - promotion_df = table_reader.read("promotion", relevant_cols=promotion_cols) - - c.create_table("store_sales", store_sales_df, persist=False) - c.create_table("item", item_df, persist=False) - c.create_table("customer", customer_df, persist=False) - c.create_table("store", store_df, persist=False) - c.create_table("date_dim", date_dim_df, persist=False) - c.create_table("customer_address", customer_address_df, persist=False) - c.create_table("promotion", promotion_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_date = f""" select min(d_date_sk) as min_d_date_sk, diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18.py b/gpu_bdb/queries/q18/gpu_bdb_query_18.py index 55022f6b..cd6b1441 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18.py @@ -24,57 +24,23 @@ left_semi_join, run_query, ) - from bdb_tools.readers import build_reader from bdb_tools.text import ( create_sentences_from_reviews, create_words_from_sentences, ) -from bdb_tools.q18_utils import find_relevant_reviews +from bdb_tools.q18_utils import ( + find_relevant_reviews, + q18_startDate, + q18_endDate, + EOL_CHAR, + read_tables +) + import numpy as np from distributed import wait - -# -------- Q18 ----------- -# -- store_sales date range -q18_startDate = "2001-05-02" -# --+90days -q18_endDate = "2001-09-02" TEMP_TABLE1 = "TEMP_TABLE1" -EOL_CHAR = "รจ" - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - - store_sales_cols = [ - "ss_store_sk", - "ss_sold_date_sk", - "ss_net_paid", - ] - date_cols = ["d_date_sk", "d_date"] - store_cols = ["s_store_sk", "s_store_name"] - - store_sales = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_dim = table_reader.read("date_dim", relevant_cols=date_cols) - store = table_reader.read("store", relevant_cols=store_cols) - - ### splitting by row groups for better parallelism - pr_table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - - product_reviews_cols = ["pr_review_date", "pr_review_content", "pr_review_sk"] - product_reviews = pr_table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols, - ) - - return store_sales, date_dim, store, product_reviews - def main(client, config): import cudf diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index 84f58d7e..6d89d7d8 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -31,56 +31,18 @@ from bdb_tools.readers import build_reader -from bdb_tools.q18_utils import find_relevant_reviews +from bdb_tools.q18_utils import ( + find_relevant_reviews, + q18_startDate, + q18_endDate, + EOL_CHAR, + read_tables +) from dask.distributed import wait - -# -------- Q18 ----------- -q18_startDate = "2001-05-02" -# --+90days -q18_endDate = "2001-09-02" - -EOL_CHAR = "รจ" - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - - store_sales_cols = [ - "ss_store_sk", - "ss_sold_date_sk", - "ss_net_paid", - ] - date_cols = ["d_date_sk", "d_date"] - store_cols = ["s_store_sk", "s_store_name"] - - store_sales = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_dim = table_reader.read("date_dim", relevant_cols=date_cols) - store = table_reader.read("store", relevant_cols=store_cols) - - ### splitting by row groups for better parallelism - pr_table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - - product_reviews_cols = ["pr_review_date", "pr_review_content", "pr_review_sk"] - product_reviews = pr_table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols, - ) - - c.create_table("store", store, persist=False) - c.create_table("store_sales", store_sales, persist=False) - c.create_table("date_dim", date_dim, persist=False) - c.create_table("product_reviews", product_reviews, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = f""" WITH temp_table1 AS diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19.py b/gpu_bdb/queries/q19/gpu_bdb_query_19.py index 8d4e29a2..6579164e 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19.py @@ -23,45 +23,16 @@ run_query, ) from bdb_tools.text import create_sentences_from_reviews, create_words_from_sentences - +from bdb_tools.q19_utils import ( + eol_char, + read_tables +) from bdb_tools.readers import build_reader from dask.distributed import Client, wait import distributed - -# -------- Q19 ----------- q19_returns_dates = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] -eol_char = "รจ" - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - date_dim_cols = ["d_week_seq", "d_date_sk", "d_date"] - date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) - store_returns_cols = ["sr_returned_date_sk", "sr_item_sk", "sr_return_quantity"] - store_returns_df = table_reader.read( - "store_returns", relevant_cols=store_returns_cols - ) - web_returns_cols = ["wr_returned_date_sk", "wr_item_sk", "wr_return_quantity"] - web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) - - ### splitting by row groups for better parallelism - pr_table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - - product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] - product_reviews = pr_table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols - ) - - return date_dim_df, store_returns_df, web_returns_df, product_reviews - def main(client, config): import cudf diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index b2c46e10..a7274809 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -34,48 +34,17 @@ from bdb_tools.readers import build_reader -from dask.distributed import wait +from bdb_tools.q19_utils import ( + eol_char, + read_tables +) +from dask.distributed import wait -# -------- Q19 ----------- q19_returns_dates_IN = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] -eol_char = "รจ" - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - date_dim_cols = ["d_week_seq", "d_date_sk", "d_date"] - date_dim_df = table_reader.read("date_dim", relevant_cols=date_dim_cols) - store_returns_cols = ["sr_returned_date_sk", "sr_item_sk", "sr_return_quantity"] - store_returns_df = table_reader.read( - "store_returns", relevant_cols=store_returns_cols - ) - web_returns_cols = ["wr_returned_date_sk", "wr_item_sk", "wr_return_quantity"] - web_returns_df = table_reader.read("web_returns", relevant_cols=web_returns_cols) - - ### splitting by row groups for better parallelism - pr_table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - - product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] - product_reviews_df = pr_table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols - ) - - c.create_table('web_returns', web_returns_df, persist=False) - c.create_table('date_dim', date_dim_df, persist=False) - c.create_table('product_reviews', product_reviews_df, persist=False) - c.create_table('store_returns', store_returns_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = f""" WITH dateFilter AS diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20.py b/gpu_bdb/queries/q20/gpu_bdb_query_20.py index e7351885..d9bcd14f 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20.py @@ -23,38 +23,13 @@ run_query, ) from bdb_tools.readers import build_reader -from bdb_tools.q20_utils import get_clusters +from bdb_tools.q20_utils import ( + get_clusters, + read_tables +) from dask import delayed from dask.distributed import wait - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_cols = [ - "ss_customer_sk", - "ss_ticket_number", - "ss_item_sk", - "ss_net_paid", - ] - store_returns_cols = [ - "sr_item_sk", - "sr_customer_sk", - "sr_ticket_number", - "sr_return_amt", - ] - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - store_returns_df = table_reader.read( - "store_returns", relevant_cols=store_returns_cols - ) - return store_sales_df, store_returns_df - - def remove_inf_and_nulls(df, column_names, value=0.0): """ Replace all nulls, inf, -inf with value column_name from df diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index 58f1ab94..22c7cedd 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -30,42 +30,13 @@ from bdb_tools.readers import build_reader -from bdb_tools.q20_utils import get_clusters - -from dask_sql import Context - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_cols = [ - "ss_customer_sk", - "ss_ticket_number", - "ss_item_sk", - "ss_net_paid", - ] - store_returns_cols = [ - "sr_item_sk", - "sr_customer_sk", - "sr_ticket_number", - "sr_return_amt", - ] - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - store_returns_df = table_reader.read( - "store_returns", relevant_cols=store_returns_cols - ) - - c.create_table("store_sales", store_sales_df, persist=False) - c.create_table("store_returns", store_returns_df, persist=False) - +from bdb_tools.q20_utils import ( + get_clusters, + read_tables +) def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = """ SELECT diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21.py b/gpu_bdb/queries/q21/gpu_bdb_query_21.py index 4d1e1217..93eb9a0c 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21.py @@ -23,6 +23,8 @@ ) from bdb_tools.merge_util import hash_merge +from bdb_tools.q21_utils import read_tables + from bdb_tools.readers import build_reader from dask.distributed import Client, wait @@ -30,60 +32,12 @@ q21_month = 1 q21_limit = 100 - -store_sales_cols = [ - "ss_item_sk", - "ss_store_sk", - "ss_customer_sk", - "ss_ticket_number", - "ss_quantity", - "ss_sold_date_sk", -] -date_cols = ["d_date_sk", "d_year", "d_moy"] -websale_cols = ["ws_item_sk", "ws_bill_customer_sk", "ws_quantity", "ws_sold_date_sk"] -sr_cols = [ - "sr_item_sk", - "sr_customer_sk", - "sr_ticket_number", - "sr_return_quantity", - "sr_returned_date_sk", -] -store_cols = ["s_store_name", "s_store_id", "s_store_sk"] -item_cols = ["i_item_id", "i_item_desc", "i_item_sk"] - -# todo: See if persisting the date table improves performence as its used all over - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) - store_retuns_df = table_reader.read("store_returns", relevant_cols=sr_cols) - store_table_df = table_reader.read("store", relevant_cols=store_cols) - item_table_df = table_reader.read("item", relevant_cols=item_cols) - - return ( - store_sales_df, - date_dim_df, - web_sales_df, - store_retuns_df, - store_table_df, - item_table_df, - ) - - def main(client, config): ( store_sales_df, date_dim_df, web_sales_df, - store_retuns_df, + store_returns_df, store_table_df, item_table_df, ) = benchmark( @@ -105,7 +59,7 @@ def main(client, config): meta=date_dim_df._meta, ).reset_index(drop=True) - part_sr = store_retuns_df.merge( + part_sr = store_returns_df.merge( d2, left_on="sr_returned_date_sk", right_on="d_date_sk", how="inner" ) diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index 626f376c..2a15b8d9 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -25,53 +25,12 @@ run_query, ) -from bdb_tools.readers import build_reader - - -store_sales_cols = [ - "ss_item_sk", - "ss_store_sk", - "ss_customer_sk", - "ss_ticket_number", - "ss_quantity", - "ss_sold_date_sk", -] -date_cols = ["d_date_sk", "d_year", "d_moy"] -websale_cols = ["ws_item_sk", "ws_bill_customer_sk", "ws_quantity", "ws_sold_date_sk"] -sr_cols = [ - "sr_item_sk", - "sr_customer_sk", - "sr_ticket_number", - "sr_return_quantity", - "sr_returned_date_sk", -] -store_cols = ["s_store_name", "s_store_id", "s_store_sk"] -item_cols = ["i_item_id", "i_item_desc", "i_item_sk"] - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - store_sales_df = table_reader.read("store_sales", relevant_cols=store_sales_cols) - date_dim_df = table_reader.read("date_dim", relevant_cols=date_cols) - web_sales_df = table_reader.read("web_sales", relevant_cols=websale_cols) - store_returns_df = table_reader.read("store_returns", relevant_cols=sr_cols) - store_table_df = table_reader.read("store", relevant_cols=store_cols) - item_table_df = table_reader.read("item", relevant_cols=item_cols) - - c.create_table("store_sales", store_sales_df, persist=False) - c.create_table("date_dim", date_dim_df, persist=False) - c.create_table("item", item_table_df, persist=False) - c.create_table("web_sales", web_sales_df, persist=False) - c.create_table("store_returns", store_returns_df, persist=False) - c.create_table("store", store_table_df, persist=False) +from bdb_tools.q21_utils import read_tables +from bdb_tools.readers import build_reader def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = """ SELECT diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22.py b/gpu_bdb/queries/q22/gpu_bdb_query_22.py index 2dfebb1f..61914568 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22.py @@ -26,7 +26,12 @@ convert_datestring_to_days, ) from bdb_tools.readers import build_reader - +from bdb_tools.q22_utils import ( + q22_date, + q22_i_current_price_min, + q22_i_current_price_max, + read_tables +) def inventory_before_after(df, date): df["inv_before"] = df["inv_quantity_on_hand"].copy() @@ -36,38 +41,7 @@ def inventory_before_after(df, date): return df -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - inv_columns = [ - "inv_item_sk", - "inv_warehouse_sk", - "inv_date_sk", - "inv_quantity_on_hand", - ] - inventory = table_reader.read("inventory", relevant_cols=inv_columns) - - item_columns = ["i_item_id", "i_current_price", "i_item_sk"] - item = table_reader.read("item", relevant_cols=item_columns) - - warehouse_columns = ["w_warehouse_sk", "w_warehouse_name"] - warehouse = table_reader.read("warehouse", relevant_cols=warehouse_columns) - - dd_columns = ["d_date_sk", "d_date"] - date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) - - return inventory, item, warehouse, date_dim - - def main(client, config): - - q22_date = "2001-05-08" - q22_i_current_price_min = 0.98 - q22_i_current_price_max = 1.5 - inventory, item, warehouse, date_dim = benchmark( read_tables, config=config, diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index 8f0d6815..9a02ab46 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -23,51 +23,20 @@ from bdb_tools.utils import ( benchmark, gpubdb_argparser, - run_query, - convert_datestring_to_days, + run_query ) -from bdb_tools.readers import build_reader - - -# -------- Q22 ----------- -q22_date = "2001-05-08" -q22_i_current_price_min = "0.98" -q22_i_current_price_max = "1.5" - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - inv_columns = [ - "inv_item_sk", - "inv_warehouse_sk", - "inv_date_sk", - "inv_quantity_on_hand", - ] - inventory = table_reader.read("inventory", relevant_cols=inv_columns) - - item_columns = ["i_item_id", "i_current_price", "i_item_sk"] - item = table_reader.read("item", relevant_cols=item_columns) - - warehouse_columns = ["w_warehouse_sk", "w_warehouse_name"] - warehouse = table_reader.read("warehouse", relevant_cols=warehouse_columns) - - dd_columns = ["d_date_sk", "d_date"] - date_dim = table_reader.read("date_dim", relevant_cols=dd_columns) - date_dim = date_dim.map_partitions(convert_datestring_to_days) - - c.create_table('inventory', inventory, persist=False) - c.create_table('item', item, persist=False) - c.create_table('warehouse', warehouse, persist=False) - c.create_table('date_dim', date_dim, persist=False) +from bdb_tools.q22_utils import ( + q22_date, + q22_i_current_price_min, + q22_i_current_price_max, + read_tables +) +from bdb_tools.readers import build_reader def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) # Filter limit in days min_date = np.datetime64(q22_date, "D").astype(int) - 30 diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23.py b/gpu_bdb/queries/q23/gpu_bdb_query_23.py index 08e1b09d..b24345ec 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23.py @@ -25,35 +25,15 @@ gpubdb_argparser, run_query, ) +from bdb_tools.q23_utils import ( + q23_year, + q23_month, + q23_coefficient, + read_tables +) from distributed import wait - -### inventory date -q23_year = 2001 -q23_month = 1 -q23_coefficient = 1.3 - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - - date_cols = ["d_date_sk", "d_year", "d_moy"] - date_df = table_reader.read("date_dim", relevant_cols=date_cols) - - inv_cols = [ - "inv_warehouse_sk", - "inv_item_sk", - "inv_date_sk", - "inv_quantity_on_hand", - ] - inv_df = table_reader.read("inventory", relevant_cols=inv_cols) - - return date_df, inv_df - - def get_iteration1(merged_inv_dates, n_workers): grouped_df = merged_inv_dates.groupby(["inv_warehouse_sk", "inv_item_sk", "d_moy"]) q23_tmp_inv_part = grouped_df.agg( diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index 020c00cd..0f6844d0 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -25,39 +25,19 @@ run_query, ) +from bdb_tools.q23_utils import ( + q23_year, + q23_month, + q23_coefficient, + read_tables +) + from bdb_tools.readers import build_reader from dask.distributed import wait - -# -------- Q23 ----------- -q23_year = 2001 -q23_month = 1 -q23_coefficient = 1.3 - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - - date_cols = ["d_date_sk", "d_year", "d_moy"] - date_df = table_reader.read("date_dim", relevant_cols=date_cols) - - inv_cols = [ - "inv_warehouse_sk", - "inv_item_sk", - "inv_date_sk", - "inv_quantity_on_hand", - ] - inv_df = table_reader.read("inventory", relevant_cols=inv_cols) - - c.create_table('inventory', inv_df, persist=False) - c.create_table('date_dim', date_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = f""" SELECT inv_warehouse_sk, diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24.py b/gpu_bdb/queries/q24/gpu_bdb_query_24.py index 5b7b2a07..bb675e78 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24.py @@ -22,43 +22,16 @@ gpubdb_argparser, run_query, ) +from bdb_tools.q24_utils import read_tables from bdb_tools.readers import build_reader from distributed import wait ### Current Implimenation Assumption ### Grouped Store sales and web sales of 1 item grouped by `date_sk` should fit in memory as number of dates is limited - ## query parameter q24_i_item_sk = 10000 -ws_cols = ["ws_item_sk", "ws_sold_date_sk", "ws_quantity"] -item_cols = ["i_item_sk", "i_current_price"] -imp_cols = [ - "imp_item_sk", - "imp_competitor_price", - "imp_start_date", - "imp_end_date", - "imp_sk", -] -ss_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_quantity"] - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - ### read tables - ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - imp_df = table_reader.read("item_marketprices", relevant_cols=imp_cols) - ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - - return ws_df, item_df, imp_df, ss_df - - def get_helper_query_table(imp_df, item_df): f_imp_df = ( imp_df.query(f"imp_item_sk == {q24_i_item_sk}", meta=imp_df._meta) diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index 3a8dbfe1..b087a3ef 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -25,40 +25,12 @@ run_query, ) -from bdb_tools.readers import build_reader - - -ws_cols = ["ws_item_sk", "ws_sold_date_sk", "ws_quantity"] -item_cols = ["i_item_sk", "i_current_price"] -imp_cols = [ - "imp_item_sk", - "imp_competitor_price", - "imp_start_date", - "imp_end_date", - "imp_sk", -] -ss_cols = ["ss_item_sk", "ss_sold_date_sk", "ss_quantity"] - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - ### read tables - ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) - item_df = table_reader.read("item", relevant_cols=item_cols) - imp_df = table_reader.read("item_marketprices", relevant_cols=imp_cols) - ss_df = table_reader.read("store_sales", relevant_cols=ss_cols) - - c.create_table("web_sales", ws_df, persist=False) - c.create_table("item", item_df, persist=False) - c.create_table("item_marketprices", imp_df, persist=False) - c.create_table("store_sales", ss_df, persist=False) +from bdb_tools.q24_utils import read_tables +from bdb_tools.readers import build_reader def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = """ WITH temp_table as diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25.py b/gpu_bdb/queries/q25/gpu_bdb_query_25.py index 5d88b643..962ab732 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25.py @@ -26,40 +26,16 @@ run_query, convert_datestring_to_days, ) +from bdb_tools.q25_utils import ( + q25_date, + N_CLUSTERS, + CLUSTER_ITERATIONS, + N_ITER, + read_tables +) from bdb_tools.readers import build_reader from dask import delayed - -# q25 parameters -Q25_DATE = "2002-01-02" -N_CLUSTERS = 8 -CLUSTER_ITERATIONS = 20 -N_ITER = 5 - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ss_cols = ["ss_customer_sk", "ss_sold_date_sk", "ss_ticket_number", "ss_net_paid"] - ws_cols = [ - "ws_bill_customer_sk", - "ws_sold_date_sk", - "ws_order_number", - "ws_net_paid", - ] - datedim_cols = ["d_date_sk", "d_date"] - - ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) - ws_ddf = table_reader.read("web_sales", relevant_cols=ws_cols, index=False) - datedim_ddf = table_reader.read("date_dim", relevant_cols=datedim_cols, index=False) - - return (ss_ddf, ws_ddf, datedim_ddf) - - def agg_count_distinct(df, group_key, counted_key, client): """Returns a Series that is the result of counting distinct instances of 'counted_key' within each 'group_key'. The series' index will have one entry per unique 'group_key' value. @@ -109,7 +85,7 @@ def main(client, config): dask_profile=config["dask_profile"], ) datedim_ddf = datedim_ddf.map_partitions(convert_datestring_to_days) - min_date = np.datetime64(Q25_DATE, "D").astype(int) + min_date = np.datetime64(q25_date, "D").astype(int) # Filter by date valid_dates_ddf = datedim_ddf[datedim_ddf["d_date"] > min_date].reset_index( drop=True diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index 3cc228f6..be8500e3 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -25,19 +25,18 @@ run_query, train_clustering_model ) -from dask import delayed - -from bdb_tools.readers import build_reader - -# -------- Q25 ----------- -# -- store_sales and web_sales date -q25_date = "2002-01-02" +from bdb_tools.q25_utils import ( + q25_date, + N_CLUSTERS, + CLUSTER_ITERATIONS, + N_ITER, + read_tables +) -N_CLUSTERS = 8 -CLUSTER_ITERATIONS = 20 -N_ITER = 5 +from dask import delayed +from bdb_tools.readers import build_reader def get_clusters(client, ml_input_df): import dask_cudf @@ -60,31 +59,6 @@ def get_clusters(client, ml_input_df): return results_dict -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ss_cols = ["ss_customer_sk", "ss_sold_date_sk", "ss_ticket_number", "ss_net_paid"] - ws_cols = [ - "ws_bill_customer_sk", - "ws_sold_date_sk", - "ws_order_number", - "ws_net_paid", - ] - datedim_cols = ["d_date_sk", "d_date"] - - ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) - ws_ddf = table_reader.read("web_sales", relevant_cols=ws_cols, index=False) - datedim_ddf = table_reader.read("date_dim", relevant_cols=datedim_cols, index=False) - - c.create_table("web_sales", ws_ddf, persist=False) - c.create_table("store_sales", ss_ddf, persist=False) - c.create_table("date_dim", datedim_ddf, persist=False) - - def agg_count_distinct(df, group_key, counted_key): """Returns a Series that is the result of counting distinct instances of 'counted_key' within each 'group_key'. The series' index will have one entry per unique 'group_key' value. @@ -102,7 +76,7 @@ def agg_count_distinct(df, group_key, counted_key): return unique_df.reset_index(drop=False) def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) q25_date = "2002-01-02" ss_join_query= f""" diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26.py b/gpu_bdb/queries/q26/gpu_bdb_query_26.py index 82596f0f..cb9b86ac 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26.py @@ -25,34 +25,17 @@ train_clustering_model, run_query, ) +from bdb_tools.q26_utils import ( + Q26_CATEGORY, + Q26_ITEM_COUNT, + N_CLUSTERS, + CLUSTER_ITERATIONS, + N_ITER, + read_tables +) from bdb_tools.readers import build_reader from dask import delayed - -# q26 parameters -Q26_CATEGORY = "Books" -Q26_ITEM_COUNT = 5 -N_CLUSTERS = 8 -CLUSTER_ITERATIONS = 20 -N_ITER = 5 - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ss_cols = ["ss_customer_sk", "ss_item_sk"] - items_cols = ["i_item_sk", "i_category", "i_class_id"] - - ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) - items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) - - return (ss_ddf, items_ddf) - - def agg_count_distinct(df, group_key, counted_key): """Returns a Series that is the result of counting distinct instances of 'counted_key' within each 'group_key'. The series' index will have one entry per unique 'group_key' value. diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 2f3aae2a..6346afb1 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -26,20 +26,19 @@ train_clustering_model ) +from bdb_tools.q26_utils import ( + Q26_CATEGORY, + Q26_ITEM_COUNT, + N_CLUSTERS, + CLUSTER_ITERATIONS, + N_ITER, + read_tables +) + from bdb_tools.readers import build_reader from dask import delayed - -# -------- Q26 ----------- -q26_i_category_IN = "Books" -q26_count_ss_item_sk = 5 - -N_CLUSTERS = 8 -CLUSTER_ITERATIONS = 20 -N_ITER = 5 - - def get_clusters(client, kmeans_input_df): import dask_cudf @@ -62,25 +61,8 @@ def get_clusters(client, kmeans_input_df): return results_dict -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - ss_cols = ["ss_customer_sk", "ss_item_sk"] - items_cols = ["i_item_sk", "i_category", "i_class_id"] - - ss_ddf = table_reader.read("store_sales", relevant_cols=ss_cols, index=False) - items_ddf = table_reader.read("item", relevant_cols=items_cols, index=False) - - c.create_table("store_sales", ss_ddf, persist=False) - c.create_table("item", items_ddf, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query = f""" SELECT @@ -105,11 +87,11 @@ def main(data_dir, client, c, config): ON ( ss.ss_item_sk = i.i_item_sk - AND i.i_category IN ('{q26_i_category_IN}') + AND i.i_category IN ('{Q26_CATEGORY}') AND ss.ss_customer_sk IS NOT NULL ) GROUP BY ss.ss_customer_sk - HAVING count(ss.ss_item_sk) > {q26_count_ss_item_sk} + HAVING count(ss.ss_item_sk) > {Q26_ITEM_COUNT} ORDER BY cid """ result = c.sql(query) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27.py b/gpu_bdb/queries/q27/gpu_bdb_query_27.py index 20da67a6..118d7aa0 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27.py @@ -31,30 +31,15 @@ from bdb_tools.readers import build_reader -from bdb_tools.q27_utils import ner_parser +from bdb_tools.q27_utils import ( + ner_parser, + q27_pr_item_sk, + EOL_CHAR, + read_tables +) from dask.distributed import wait - -# -------- Q27 ----------- -q27_pr_item_sk = 10002 -EOL_CHAR = "." - - -def read_tables(config): - ### splitting by row groups for better parallelism - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] - product_reviews_df = table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols - ) - return product_reviews_df - - def main(client, config): import dask_cudf diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index a707d4c7..3ad610ce 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -30,33 +30,17 @@ from bdb_tools.readers import build_reader -from bdb_tools.q27_utils import ner_parser +from bdb_tools.q27_utils import ( + ner_parser, + q27_pr_item_sk, + EOL_CHAR, + read_tables +) from dask.distributed import wait - -# -------- Q27 ----------- -q27_pr_item_sk = 10002 -EOL_CHAR = "." - - -def read_tables(data_dir, c, config): - ### splitting by row groups for better parallelism - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - product_reviews_cols = ["pr_item_sk", "pr_review_content", "pr_review_sk"] - product_reviews_df = table_reader.read( - "product_reviews", relevant_cols=product_reviews_cols - ) - - c.create_table("product_reviews", product_reviews_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) import dask_cudf diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28.py b/gpu_bdb/queries/q28/gpu_bdb_query_28.py index 4adf3cc3..45558583 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28.py @@ -35,25 +35,10 @@ ) from bdb_tools.readers import build_reader -from bdb_tools.q28_utils import post_etl_processing - - -def read_tables(config): - ### splitting by row groups for better parallelism - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - - columns = [ - "pr_review_content", - "pr_review_rating", - "pr_review_sk", - ] - ret = table_reader.read("product_reviews", relevant_cols=columns) - return ret - +from bdb_tools.q28_utils import ( + post_etl_processing, + read_tables +) def main(client, config): q_st = time.time() diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 4b1bd773..480f4e57 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -24,29 +24,13 @@ from bdb_tools.readers import build_reader -from bdb_tools.q28_utils import post_etl_processing - - -def read_tables(data_dir, c, config): - ### splitting by row groups for better parallelism - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=True, - ) - - columns = [ - "pr_review_content", - "pr_review_rating", - "pr_review_sk", - ] - pr_df = table_reader.read("product_reviews", relevant_cols=columns) - - c.create_table("product_reviews", pr_df, persist=False) - +from bdb_tools.q28_utils import ( + post_etl_processing, + read_tables +) def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) # 10 % of data query1 = """ diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29.py b/gpu_bdb/queries/q29/gpu_bdb_query_29.py index 88ead76f..a8726c83 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29.py @@ -21,6 +21,10 @@ gpubdb_argparser, run_query, ) +from bdb_tools.q29_utils import ( + q29_limit, + read_tables +) from bdb_tools.readers import build_reader from bdb_tools.utils import benchmark from distributed import wait @@ -39,25 +43,8 @@ ### Scalabilty problems # * The ws_item_join table after distincts has `48M` rows, can cause problems on bigger scale factors - -# -------- Q29 ----------- -q29_limit = 100 q29_session_timeout_inSec = 3600 - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - item_cols = ["i_item_sk", "i_category_id"] - item_df = table_reader.read("item", relevant_cols=item_cols) - - ws_cols = ["ws_order_number", "ws_item_sk"] - ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) - - return item_df, ws_df - - ### # Select t1.i_category_id AS category_id_1 , t2.i_category_id AS category_id_2 # FROM ( diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index d11e109d..5b2a6762 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -25,31 +25,17 @@ run_query, ) +from bdb_tools.q29_utils import ( + q29_limit, + read_tables +) + from bdb_tools.readers import build_reader from dask.distributed import wait - -# -------- Q29 ----------- -q29_limit = 100 - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], basepath=config["data_dir"], - ) - item_cols = ["i_item_sk", "i_category_id"] - item_df = table_reader.read("item", relevant_cols=item_cols) - - ws_cols = ["ws_order_number", "ws_item_sk"] - ws_df = table_reader.read("web_sales", relevant_cols=ws_cols) - - c.create_table('item', item_df, persist=False) - c.create_table('web_sales', ws_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) n_workers = len(client.scheduler_info()["workers"]) join_query = """ diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30.py b/gpu_bdb/queries/q30/gpu_bdb_query_30.py index 5f9eaac5..07b6a0bf 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30.py @@ -23,6 +23,11 @@ gpubdb_argparser, run_query, ) +from bdb_tools.q30_utils import ( + q30_session_timeout_inSec, + q30_limit, + read_tables +) from bdb_tools.readers import build_reader from bdb_tools.sessionization import get_session_id, get_distinct_sessions, get_pairs @@ -35,25 +40,6 @@ # The bottleneck of current implementation is `set-index`, once ucx is working correctly # it should go away - -### session timeout in secs -q30_session_timeout_inSec = 3600 -### query output limit -q30_limit = 40 - - -def read_tables(config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_cols = ["i_category_id", "i_item_sk"] - item_df = table_reader.read("item", relevant_cols=item_cols) - return item_df - - def pre_repartition_task(wcs_fn, f_item_df): """ Runs the pre-repartition task diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 7468dec8..520383e3 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -30,38 +30,18 @@ get_pairs ) +from bdb_tools.q30_utils import ( + q30_session_timeout_inSec, + q30_limit, + read_tables +) + from bdb_tools.readers import build_reader from dask.distributed import wait - -# -------- Q30 ----------- -# session timeout in secs -q30_session_timeout_inSec = 3600 -# query output limit -q30_limit = 40 - - - -def read_tables(data_dir, c, config): - table_reader = build_reader( - data_format=config["file_format"], - basepath=config["data_dir"], - split_row_groups=config["split_row_groups"], - ) - - item_cols = ["i_category_id", "i_item_sk"] - item_df = table_reader.read("item", relevant_cols=item_cols) - - wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] - wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - - c.create_table('web_clickstreams', wcs_df, persist=False) - c.create_table('item', item_df, persist=False) - - def main(data_dir, client, c, config): - benchmark(read_tables, data_dir, c, config, dask_profile=config["dask_profile"]) + benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) query_1 = """ SELECT i_item_sk, From 6470303a31608e5b138ffb423529681636aa0adf Mon Sep 17 00:00:00 2001 From: sft-managed Date: Thu, 20 Jan 2022 06:31:00 +0000 Subject: [PATCH 37/51] Update copyrights --- gpu_bdb/queries/q01/gpu_bdb_query_01.py | 3 +-- gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py | 1 - gpu_bdb/queries/q02/gpu_bdb_query_02.py | 3 +-- gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 2 -- gpu_bdb/queries/q03/gpu_bdb_query_03.py | 3 +-- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 2 -- gpu_bdb/queries/q04/gpu_bdb_query_04.py | 4 +--- gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 2 -- gpu_bdb/queries/q05/gpu_bdb_query_05.py | 1 - gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py | 1 - gpu_bdb/queries/q06/gpu_bdb_query_06.py | 3 +-- gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py | 2 -- gpu_bdb/queries/q07/gpu_bdb_query_07.py | 4 +--- gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py | 2 -- gpu_bdb/queries/q08/gpu_bdb_query_08.py | 3 +-- gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 2 -- gpu_bdb/queries/q09/gpu_bdb_query_09.py | 3 +-- gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py | 2 -- gpu_bdb/queries/q10/gpu_bdb_query_10.py | 3 +-- gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py | 2 -- gpu_bdb/queries/q11/gpu_bdb_query_11.py | 3 +-- gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py | 2 -- gpu_bdb/queries/q12/gpu_bdb_query_12.py | 3 +-- gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py | 2 -- gpu_bdb/queries/q13/gpu_bdb_query_13.py | 3 +-- gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py | 2 -- gpu_bdb/queries/q14/gpu_bdb_query_14.py | 3 +-- gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py | 2 -- gpu_bdb/queries/q15/gpu_bdb_query_15.py | 3 +-- gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py | 2 -- gpu_bdb/queries/q16/gpu_bdb_query_16.py | 3 +-- gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py | 2 -- gpu_bdb/queries/q17/gpu_bdb_query_17.py | 4 ++-- gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py | 2 -- gpu_bdb/queries/q18/gpu_bdb_query_18.py | 1 - gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py | 2 -- gpu_bdb/queries/q19/gpu_bdb_query_19.py | 3 +-- gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py | 2 -- gpu_bdb/queries/q20/gpu_bdb_query_20.py | 1 - gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py | 2 -- gpu_bdb/queries/q21/gpu_bdb_query_21.py | 3 +-- gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py | 2 -- gpu_bdb/queries/q22/gpu_bdb_query_22.py | 3 +-- gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py | 2 -- gpu_bdb/queries/q23/gpu_bdb_query_23.py | 4 +--- gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py | 2 -- gpu_bdb/queries/q24/gpu_bdb_query_24.py | 3 +-- gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py | 2 -- gpu_bdb/queries/q25/gpu_bdb_query_25.py | 3 +-- gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py | 2 -- gpu_bdb/queries/q26/gpu_bdb_query_26.py | 3 +-- gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py | 2 -- gpu_bdb/queries/q27/gpu_bdb_query_27.py | 2 -- gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py | 2 -- gpu_bdb/queries/q28/gpu_bdb_query_28.py | 1 - gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py | 2 -- gpu_bdb/queries/q29/gpu_bdb_query_29.py | 3 +-- gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 2 -- gpu_bdb/queries/q30/gpu_bdb_query_30.py | 3 +-- gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 2 -- 60 files changed, 26 insertions(+), 117 deletions(-) diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01.py b/gpu_bdb/queries/q01/gpu_bdb_query_01.py index 912af07d..2d7c28b3 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -15,7 +15,6 @@ # from bdb_tools.utils import benchmark, gpubdb_argparser, run_query -from bdb_tools.readers import build_reader from bdb_tools.q01_utils import ( q01_viewed_together_count, diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index 504497ea..335dcf23 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -24,7 +24,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q01_utils import ( q01_viewed_together_count, diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02.py b/gpu_bdb/queries/q02/gpu_bdb_query_02.py index 8b6ac390..8c9c4516 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -19,7 +19,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.sessionization import get_distinct_sessions from bdb_tools.q02_utils import ( q02_item_sk, diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index ef82a263..38f520e6 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -25,8 +25,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.sessionization import get_distinct_sessions from bdb_tools.q02_utils import ( diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03.py b/gpu_bdb/queries/q03/gpu_bdb_query_03.py index 2404e357..a343faac 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -21,7 +21,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q03_utils import ( apply_find_items_viewed, diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 3a0cbe33..866e90a9 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -22,8 +22,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q03_utils import ( apply_find_items_viewed, q03_days_in_sec_before_purchase, diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04.py b/gpu_bdb/queries/q04/gpu_bdb_query_04.py index 9134f99c..bc4ff1d2 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -22,8 +22,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.sessionization import get_sessions from bdb_tools.q04_utils import ( diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index e37468b4..17a02159 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -27,8 +27,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q04_utils import ( abandonedShoppingCarts, reduction_function, diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05.py b/gpu_bdb/queries/q05/gpu_bdb_query_05.py index f1e3a4f0..c3b7fe87 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05.py @@ -23,7 +23,6 @@ run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q05_utils import ( build_and_predict_model, read_tables diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py index 6ad5ef1c..e0a628ca 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05_dask_sql.py @@ -23,7 +23,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q05_utils import ( build_and_predict_model, read_tables diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06.py b/gpu_bdb/queries/q06/gpu_bdb_query_06.py index d621cf2e..60e79017 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -22,7 +22,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from distributed import wait from bdb_tools.q06_utils import ( diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index a5ea5328..7f0b51f4 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -25,8 +25,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q06_utils import ( q06_LIMIT, q06_YEAR, diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07.py b/gpu_bdb/queries/q07/gpu_bdb_query_07.py index 6f0350c5..688269b2 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -22,8 +22,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q07_utils import read_tables q07_HIGHER_PRICE_RATIO = 1.2 diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index 3133500c..0efe38d7 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -26,8 +26,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q07_utils import read_tables def main(data_dir, client, c, config): diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08.py b/gpu_bdb/queries/q08/gpu_bdb_query_08.py index f0c59fb5..fef5ff83 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08.py @@ -21,9 +21,8 @@ benchmark, gpubdb_argparser, run_query, - convert_datestring_to_days, + convert_datestring_to_days ) -from bdb_tools.readers import build_reader from bdb_tools.merge_util import hash_merge from bdb_tools.q08_utils import ( get_sessions, diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 38f8b086..9f2641a3 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -24,8 +24,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q08_utils import ( get_sessions, get_unique_sales_keys_from_sessions, diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09.py b/gpu_bdb/queries/q09/gpu_bdb_query_09.py index 60ea0c2b..8ac21a75 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -22,7 +22,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q09_utils import ( q09_year, diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index c1243ec0..fc1b6fb4 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -25,8 +25,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q09_utils import ( q09_year, q09_part1_ca_country, diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10.py b/gpu_bdb/queries/q10/gpu_bdb_query_10.py index 2cb59378..f6bc5e6d 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -23,7 +23,6 @@ run_query, ) from bdb_tools.text import create_sentences_from_reviews, create_words_from_sentences -from bdb_tools.readers import build_reader from bdb_tools.q10_utils import ( eol_char, read_tables diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index dd8d83c0..702199b4 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -32,8 +32,6 @@ create_words_from_sentences ) -from bdb_tools.readers import build_reader - from bdb_tools.q10_utils import ( eol_char, read_tables diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11.py b/gpu_bdb/queries/q11/gpu_bdb_query_11.py index bc0199b1..dd5879c9 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -23,7 +23,6 @@ run_query, convert_datestring_to_days, ) -from bdb_tools.readers import build_reader from bdb_tools.q11_utils import read_tables diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index 9e69cbff..0b0bd8c3 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -26,8 +26,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q11_utils import read_tables def main(data_dir, client, c, config): diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12.py b/gpu_bdb/queries/q12/gpu_bdb_query_12.py index 8c17e8e2..2bee6465 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -23,7 +23,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q12_utils import read_tables from distributed import wait diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index b55471e3..108d1602 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -25,8 +25,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q12_utils import read_tables q12_i_category_IN = "'Books', 'Electronics'" diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13.py b/gpu_bdb/queries/q13/gpu_bdb_query_13.py index 316c4a96..fba18806 100755 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -21,7 +21,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q13_utils import read_tables from distributed import wait diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index c9c7da30..20177648 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -26,8 +26,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q13_utils import read_tables from dask.distributed import wait diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14.py b/gpu_bdb/queries/q14/gpu_bdb_query_14.py index 6cb7a605..7112f832 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -25,7 +25,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q14_utils import read_tables def main(client, config): diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index aaea754f..120175e8 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -25,8 +25,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q14_utils import read_tables def main(data_dir, client, c, config): diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15.py b/gpu_bdb/queries/q15/gpu_bdb_query_15.py index 7249051f..3a13fada 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -24,7 +24,6 @@ run_query, convert_datestring_to_days, ) -from bdb_tools.readers import build_reader from bdb_tools.q15_utils import ( q15_startDate, q15_endDate, diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index 84783fd9..d990abbb 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -25,8 +25,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q15_utils import ( q15_startDate, q15_endDate, diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16.py b/gpu_bdb/queries/q16/gpu_bdb_query_16.py index be4bd598..a06bc86a 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -16,7 +16,6 @@ import sys - from bdb_tools.utils import ( benchmark, gpubdb_argparser, diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 38618dc2..4d9da53a 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -27,8 +27,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q16_utils import read_tables def main(data_dir, client, c, config): diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17.py b/gpu_bdb/queries/q17/gpu_bdb_query_17.py index 1bd4131b..6d7f860d 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -22,7 +22,7 @@ gpubdb_argparser, left_semi_join, run_query, -) + ) from bdb_tools.q17_utils import ( q17_gmt_offset, q17_year, diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 58d44ba1..411879cd 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -25,8 +25,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q17_utils import ( q17_gmt_offset, q17_year, diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18.py b/gpu_bdb/queries/q18/gpu_bdb_query_18.py index cd6b1441..cabe6f2d 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18.py @@ -24,7 +24,6 @@ left_semi_join, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.text import ( create_sentences_from_reviews, create_words_from_sentences, diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py index 6d89d7d8..4a1eba70 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18_dask_sql.py @@ -29,8 +29,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q18_utils import ( find_relevant_reviews, q18_startDate, diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19.py b/gpu_bdb/queries/q19/gpu_bdb_query_19.py index 6579164e..827c6a7a 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -28,7 +28,6 @@ read_tables ) -from bdb_tools.readers import build_reader from dask.distributed import Client, wait import distributed diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index a7274809..ba7a66fe 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -32,8 +32,6 @@ create_words_from_sentences ) -from bdb_tools.readers import build_reader - from bdb_tools.q19_utils import ( eol_char, read_tables diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20.py b/gpu_bdb/queries/q20/gpu_bdb_query_20.py index d9bcd14f..8559f101 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20.py @@ -22,7 +22,6 @@ train_clustering_model, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q20_utils import ( get_clusters, read_tables diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index 22c7cedd..71f275f9 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -28,8 +28,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q20_utils import ( get_clusters, read_tables diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21.py b/gpu_bdb/queries/q21/gpu_bdb_query_21.py index 93eb9a0c..7cb4c8a8 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -25,7 +25,6 @@ from bdb_tools.q21_utils import read_tables -from bdb_tools.readers import build_reader from dask.distributed import Client, wait q21_year = 2003 diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index 2a15b8d9..cb925cd9 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -27,8 +27,6 @@ from bdb_tools.q21_utils import read_tables -from bdb_tools.readers import build_reader - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22.py b/gpu_bdb/queries/q22/gpu_bdb_query_22.py index 61914568..082979a7 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -25,7 +25,6 @@ run_query, convert_datestring_to_days, ) -from bdb_tools.readers import build_reader from bdb_tools.q22_utils import ( q22_date, q22_i_current_price_min, diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index 9a02ab46..67834725 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -33,8 +33,6 @@ read_tables ) -from bdb_tools.readers import build_reader - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23.py b/gpu_bdb/queries/q23/gpu_bdb_query_23.py index b24345ec..4e9621c4 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -18,8 +18,6 @@ import sys import rmm - -from bdb_tools.readers import build_reader from bdb_tools.utils import ( benchmark, gpubdb_argparser, diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index 0f6844d0..b7d9f943 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -32,8 +32,6 @@ read_tables ) -from bdb_tools.readers import build_reader - from dask.distributed import wait def main(data_dir, client, c, config): diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24.py b/gpu_bdb/queries/q24/gpu_bdb_query_24.py index bb675e78..1aedd1ab 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -23,7 +23,6 @@ run_query, ) from bdb_tools.q24_utils import read_tables -from bdb_tools.readers import build_reader from distributed import wait ### Current Implimenation Assumption diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index b087a3ef..00c06027 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -27,8 +27,6 @@ from bdb_tools.q24_utils import read_tables -from bdb_tools.readers import build_reader - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25.py b/gpu_bdb/queries/q25/gpu_bdb_query_25.py index 962ab732..e0d64816 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -33,7 +33,6 @@ N_ITER, read_tables ) -from bdb_tools.readers import build_reader from dask import delayed def agg_count_distinct(df, group_key, counted_key, client): diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index be8500e3..85eb9dcf 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -36,8 +36,6 @@ from dask import delayed -from bdb_tools.readers import build_reader - def get_clusters(client, ml_input_df): import dask_cudf diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26.py b/gpu_bdb/queries/q26/gpu_bdb_query_26.py index cb9b86ac..5169cb74 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -33,7 +33,6 @@ N_ITER, read_tables ) -from bdb_tools.readers import build_reader from dask import delayed def agg_count_distinct(df, group_key, counted_key): diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index 6346afb1..d452c2a9 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -35,8 +35,6 @@ read_tables ) -from bdb_tools.readers import build_reader - from dask import delayed def get_clusters(client, kmeans_input_df): diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27.py b/gpu_bdb/queries/q27/gpu_bdb_query_27.py index 118d7aa0..afa939a8 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27.py @@ -29,8 +29,6 @@ create_words_from_sentences ) -from bdb_tools.readers import build_reader - from bdb_tools.q27_utils import ( ner_parser, q27_pr_item_sk, diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index 3ad610ce..52b20fef 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -28,8 +28,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q27_utils import ( ner_parser, q27_pr_item_sk, diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28.py b/gpu_bdb/queries/q28/gpu_bdb_query_28.py index 45558583..1ec0ac63 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28.py @@ -33,7 +33,6 @@ gpubdb_argparser, run_query, ) -from bdb_tools.readers import build_reader from bdb_tools.q28_utils import ( post_etl_processing, diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py index 480f4e57..aa6c5e76 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28_dask_sql.py @@ -22,8 +22,6 @@ run_query, ) -from bdb_tools.readers import build_reader - from bdb_tools.q28_utils import ( post_etl_processing, read_tables diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29.py b/gpu_bdb/queries/q29/gpu_bdb_query_29.py index a8726c83..dbc6fa6b 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -25,7 +25,6 @@ q29_limit, read_tables ) -from bdb_tools.readers import build_reader from bdb_tools.utils import benchmark from distributed import wait diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 5b2a6762..75f409d4 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -30,8 +30,6 @@ read_tables ) -from bdb_tools.readers import build_reader - from dask.distributed import wait def main(data_dir, client, c, config): diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30.py b/gpu_bdb/queries/q30/gpu_bdb_query_30.py index 07b6a0bf..af00d031 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30.py @@ -1,5 +1,5 @@ # -# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# Copyright (c) 2019-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -28,7 +28,6 @@ q30_limit, read_tables ) -from bdb_tools.readers import build_reader from bdb_tools.sessionization import get_session_id, get_distinct_sessions, get_pairs from dask import delayed diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 520383e3..18050907 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -36,8 +36,6 @@ read_tables ) -from bdb_tools.readers import build_reader - from dask.distributed import wait def main(data_dir, client, c, config): From 3e9d688bd85db0c8997e2d4271d833affb111a3d Mon Sep 17 00:00:00 2001 From: sft-managed Date: Thu, 20 Jan 2022 20:04:17 +0000 Subject: [PATCH 38/51] Remove unused imports --- gpu_bdb/bdb_tools/q03_utils.py | 4 ---- gpu_bdb/bdb_tools/q08_utils.py | 4 ++-- gpu_bdb/bdb_tools/q28_utils.py | 5 ----- gpu_bdb/queries/q01/gpu_bdb_query_01.py | 2 -- gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py | 3 --- gpu_bdb/queries/q02/gpu_bdb_query_02.py | 7 ------- gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py | 6 ------ gpu_bdb/queries/q03/gpu_bdb_query_03.py | 12 +++--------- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 2 -- gpu_bdb/queries/q04/gpu_bdb_query_04.py | 8 +------- gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py | 4 ---- gpu_bdb/queries/q05/gpu_bdb_query_05.py | 14 ++++---------- gpu_bdb/queries/q06/gpu_bdb_query_06.py | 6 ------ gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py | 3 --- gpu_bdb/queries/q07/gpu_bdb_query_07.py | 5 ----- gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py | 4 ---- gpu_bdb/queries/q08/gpu_bdb_query_08.py | 8 +++----- gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py | 2 -- gpu_bdb/queries/q09/gpu_bdb_query_09.py | 6 +----- gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py | 3 --- gpu_bdb/queries/q10/gpu_bdb_query_10.py | 15 ++++----------- gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py | 1 - gpu_bdb/queries/q11/gpu_bdb_query_11.py | 6 +----- gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py | 3 --- gpu_bdb/queries/q12/gpu_bdb_query_12.py | 8 +++----- gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py | 3 --- gpu_bdb/queries/q13/gpu_bdb_query_13.py | 4 ---- gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py | 4 ---- gpu_bdb/queries/q14/gpu_bdb_query_14.py | 7 +------ gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py | 3 --- gpu_bdb/queries/q15/gpu_bdb_query_15.py | 7 ------- gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py | 3 --- gpu_bdb/queries/q16/gpu_bdb_query_16.py | 5 +---- gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py | 3 --- gpu_bdb/queries/q17/gpu_bdb_query_17.py | 6 +----- gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py | 3 --- gpu_bdb/queries/q18/gpu_bdb_query_18.py | 8 ++------ gpu_bdb/queries/q19/gpu_bdb_query_19.py | 11 ++++------- gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py | 1 - gpu_bdb/queries/q20/gpu_bdb_query_20.py | 4 ---- gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py | 5 ----- gpu_bdb/queries/q21/gpu_bdb_query_21.py | 6 +----- gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py | 3 --- gpu_bdb/queries/q22/gpu_bdb_query_22.py | 5 ----- gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py | 2 -- gpu_bdb/queries/q23/gpu_bdb_query_23.py | 6 ------ gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py | 3 --- gpu_bdb/queries/q24/gpu_bdb_query_24.py | 5 ----- gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py | 3 --- gpu_bdb/queries/q25/gpu_bdb_query_25.py | 9 ++------- gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py | 4 ---- gpu_bdb/queries/q26/gpu_bdb_query_26.py | 7 ------- gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py | 3 --- gpu_bdb/queries/q27/gpu_bdb_query_27.py | 7 +------ gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py | 2 -- gpu_bdb/queries/q28/gpu_bdb_query_28.py | 15 --------------- gpu_bdb/queries/q29/gpu_bdb_query_29.py | 6 ------ gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py | 3 --- gpu_bdb/queries/q30/gpu_bdb_query_30.py | 11 ++++------- gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py | 3 --- 60 files changed, 39 insertions(+), 282 deletions(-) diff --git a/gpu_bdb/bdb_tools/q03_utils.py b/gpu_bdb/bdb_tools/q03_utils.py index a02543c5..472685ca 100644 --- a/gpu_bdb/bdb_tools/q03_utils.py +++ b/gpu_bdb/bdb_tools/q03_utils.py @@ -14,9 +14,6 @@ # limitations under the License. # -import os -import sys - import cudf from numba import cuda @@ -63,7 +60,6 @@ def find_items_viewed_before_purchase_kernel( as defined by the configuration of this query. """ i = cuda.grid(1) - relevant_item = q03_purchased_item_IN if i < (relevant_idx_col.size): # boundary guard # every relevant row gets N rows in the output, so we need to map the indexes diff --git a/gpu_bdb/bdb_tools/q08_utils.py b/gpu_bdb/bdb_tools/q08_utils.py index 95a32cf8..2a220cb1 100644 --- a/gpu_bdb/bdb_tools/q08_utils.py +++ b/gpu_bdb/bdb_tools/q08_utils.py @@ -70,7 +70,7 @@ def get_session_id_from_session_boundary(session_change_df, last_session_len): try: session_len.iloc[-1] = last_session_len - except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 + except (AssertionError, IndexError): # IndexError in numba >= 0.48 session_len = cudf.Series([]) session_id_final_series = ( @@ -96,7 +96,7 @@ def get_session_id(df): session_change_df = df[df["session_change_flag"]].reset_index(drop=True) try: last_session_len = len(df) - session_change_df["t_index"].iloc[-1] - except (AssertionError, IndexError) as e: # IndexError in numba >= 0.48 + except (AssertionError, IndexError): # IndexError in numba >= 0.48 last_session_len = 0 session_ids = get_session_id_from_session_boundary( diff --git a/gpu_bdb/bdb_tools/q28_utils.py b/gpu_bdb/bdb_tools/q28_utils.py index 140aef6a..c594dae9 100644 --- a/gpu_bdb/bdb_tools/q28_utils.py +++ b/gpu_bdb/bdb_tools/q28_utils.py @@ -14,8 +14,6 @@ # limitations under the License. # -import time - import numpy as np import cupy as cp import cupy @@ -273,7 +271,6 @@ def post_etl_processing(client, train_data, test_data): model.fit(X_train, y_train) ### this regression seems to be coming from here - test_pred_st = time.time() y_hat = model.predict(X_test).persist() # Compute distributed performance metrics @@ -286,10 +283,8 @@ def post_etl_processing(client, train_data, test_data): cmat = confusion_matrix(client, y_test, y_hat) print("Confusion Matrix: " + str(cmat)) - metric_et = time.time() # Place results back in original Dataframe - ddh = DistributedDataHandler.create(y_hat) test_preds = to_dask_cudf( [client.submit(cudf.Series, part) for w, part in ddh.gpu_futures] diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01.py b/gpu_bdb/queries/q01/gpu_bdb_query_01.py index 2d7c28b3..f0838ff2 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01.py @@ -145,8 +145,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index 335dcf23..6cc4428d 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -14,10 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -import os from bdb_tools.utils import ( benchmark, diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02.py b/gpu_bdb/queries/q02/gpu_bdb_query_02.py index 8c9c4516..c6c11e40 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02.py @@ -22,7 +22,6 @@ from bdb_tools.sessionization import get_distinct_sessions from bdb_tools.q02_utils import ( q02_item_sk, - q02_MAX_ITEMS_PER_BASKET, q02_limit, q02_session_timeout_inSec, read_tables @@ -64,10 +63,6 @@ def reduction_function(df, q02_session_timeout_inSec): return grouped_df - wcs_df = table_reader.read("web_clickstreams", relevant_cols=wcs_cols) - return wcs_df - - def pre_repartition_task(wcs_df): f_wcs_df = wcs_df[ @@ -141,8 +136,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py index 38f520e6..38c1668f 100755 --- a/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py +++ b/gpu_bdb/queries/q02/gpu_bdb_query_02_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( @@ -34,9 +31,6 @@ read_tables ) -from dask.distributed import wait - - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03.py b/gpu_bdb/queries/q03/gpu_bdb_query_03.py index a343faac..b0e0562f 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03.py @@ -16,6 +16,9 @@ import os +import cudf +import dask_cudf + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -24,8 +27,6 @@ from bdb_tools.q03_utils import ( apply_find_items_viewed, - q03_days_in_sec_before_purchase, - q03_views_before_purchase, q03_purchased_item_IN, q03_limit, read_tables @@ -37,11 +38,9 @@ import glob from dask import delayed - q03_purchased_item_category_IN = [2, 3] def get_wcs_minima(config): - import dask_cudf wcs_df = dask_cudf.read_parquet( os.path.join(config["data_dir"], "web_clickstreams/*.parquet"), @@ -55,7 +54,6 @@ def get_wcs_minima(config): def pre_repartition_task(wcs_fn, item_df, wcs_tstamp_min): - import cudf wcs_cols = [ "wcs_user_sk", @@ -109,8 +107,6 @@ def reduction_function(df, item_df_filtered): def main(client, config): - import dask_cudf - import cudf item_df = benchmark( read_tables, @@ -190,8 +186,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index 866e90a9..e7f200ac 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -24,8 +24,6 @@ from bdb_tools.q03_utils import ( apply_find_items_viewed, - q03_days_in_sec_before_purchase, - q03_views_before_purchase, q03_purchased_item_IN, q03_limit, read_tables diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04.py b/gpu_bdb/queries/q04/gpu_bdb_query_04.py index bc4ff1d2..0e798bd7 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04.py @@ -14,7 +14,7 @@ # limitations under the License. # -import sys +import cudf from bdb_tools.utils import ( benchmark, @@ -22,10 +22,7 @@ run_query, ) -from bdb_tools.sessionization import get_sessions - from bdb_tools.q04_utils import ( - abandonedShoppingCarts, reduction_function, read_tables ) @@ -34,7 +31,6 @@ q04_session_timeout_inSec = 3600 def main(client, config): - import cudf wp, wcs_df = benchmark( read_tables, @@ -93,8 +89,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py index 17a02159..3af8ef10 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - import cudf from bdb_tools.cluster_startup import attach_to_cluster @@ -28,7 +25,6 @@ ) from bdb_tools.q04_utils import ( - abandonedShoppingCarts, reduction_function, read_tables ) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05.py b/gpu_bdb/queries/q05/gpu_bdb_query_05.py index c3b7fe87..ce94f09d 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05.py @@ -17,6 +17,9 @@ import os import glob +import cudf +import dask_cudf + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -25,13 +28,12 @@ from bdb_tools.q05_utils import ( build_and_predict_model, + wcs_columns, read_tables ) -import cupy as cp import numpy as np from dask import delayed -import dask import pandas as pd # @@ -40,13 +42,10 @@ COLLEGE_ED_STRS = ["Advanced Degree", "College", "4 yr Degree", "2 yr Degree"] Q05_I_CATEGORY = "Books" -wcs_columns = ["wcs_item_sk", "wcs_user_sk"] - def get_groupby_results(file_list, item_df): """ Functionial approach for better scaling """ - import cudf sum_by_cat_ddf = None for fn in file_list: @@ -91,8 +90,6 @@ def get_groupby_results(file_list, item_df): def main(client, config): - import cudf - import dask_cudf item_ddf, customer_ddf, customer_dem_ddf = benchmark( read_tables, @@ -197,9 +194,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf - import cuml config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06.py b/gpu_bdb/queries/q06/gpu_bdb_query_06.py index 60e79017..9e3e9ff7 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06.py @@ -14,15 +14,11 @@ # limitations under the License. # -import sys - - from bdb_tools.utils import ( benchmark, gpubdb_argparser, run_query, ) -from distributed import wait from bdb_tools.q06_utils import ( q06_YEAR, @@ -209,8 +205,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py index 7f0b51f4..736319c4 100755 --- a/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py +++ b/gpu_bdb/queries/q06/gpu_bdb_query_06_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07.py b/gpu_bdb/queries/q07/gpu_bdb_query_07.py index 688269b2..89e8903a 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys - - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -120,8 +117,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py index 0efe38d7..8ff1e73f 100755 --- a/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py +++ b/gpu_bdb/queries/q07/gpu_bdb_query_07_dask_sql.py @@ -14,11 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -from dask.distributed import Client -import os from bdb_tools.utils import ( benchmark, diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08.py b/gpu_bdb/queries/q08/gpu_bdb_query_08.py index fef5ff83..451cbe9e 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08.py @@ -17,6 +17,9 @@ import os import glob +import cudf +import dask_cudf + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -38,7 +41,6 @@ from dask import delayed def etl_wcs(wcs_fn, filtered_date_df, web_page_df): - import cudf filtered_date_df = filtered_date_df web_page_df = web_page_df @@ -91,8 +93,6 @@ def reduction_function(df, REVIEW_CAT_CODE): def main(client, config): - import cudf - import dask_cudf (date_dim_df, web_page_df, web_sales_df) = benchmark( read_tables, @@ -205,8 +205,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py index 9f2641a3..6a85bc1c 100755 --- a/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py +++ b/gpu_bdb/queries/q08/gpu_bdb_query_08_dask_sql.py @@ -14,8 +14,6 @@ # limitations under the License. # -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09.py b/gpu_bdb/queries/q09/gpu_bdb_query_09.py index 8ac21a75..36a3bf80 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09.py @@ -14,8 +14,7 @@ # limitations under the License. # -from dask.distributed import Client -import sys +import cudf from bdb_tools.utils import ( benchmark, @@ -50,7 +49,6 @@ ) def main(client, config): - import cudf q09_part1_ca_state_IN = "KY", "GA", "NM" q09_part2_ca_state_IN = "MT", "OR", "IN" @@ -169,8 +167,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index fc1b6fb4..d13835c1 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -14,10 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -import os from bdb_tools.utils import ( benchmark, diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10.py b/gpu_bdb/queries/q10/gpu_bdb_query_10.py index f6bc5e6d..4cc3e833 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10.py @@ -14,9 +14,11 @@ # limitations under the License. # -import sys import os +import cudf +import dask_cudf + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -28,14 +30,9 @@ read_tables ) -import rmm -import cupy as cp -import distributed - -from dask.distributed import Client, wait +from dask.distributed import wait def load_sentiment_words(filename, sentiment): - import cudf with open(filename) as fh: sentiment_words = list(map(str.strip, fh.readlines())) @@ -48,8 +45,6 @@ def load_sentiment_words(filename, sentiment): def main(client, config): - import cudf - import dask_cudf product_reviews_df = benchmark( read_tables, @@ -131,8 +126,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py index 702199b4..64dba763 100755 --- a/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py +++ b/gpu_bdb/queries/q10/gpu_bdb_query_10_dask_sql.py @@ -14,7 +14,6 @@ # limitations under the License. # -import sys import os import dask_cudf diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11.py b/gpu_bdb/queries/q11/gpu_bdb_query_11.py index dd5879c9..224daf40 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11.py @@ -14,8 +14,7 @@ # limitations under the License. # -import sys - +import cudf from bdb_tools.utils import ( benchmark, @@ -32,7 +31,6 @@ q11_end_date = "2003-02-02" def main(client, config): - import cudf pr_df, ws_df, date_df = benchmark( read_tables, @@ -95,8 +93,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py index 0b0bd8c3..b5d41715 100755 --- a/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py +++ b/gpu_bdb/queries/q11/gpu_bdb_query_11_dask_sql.py @@ -14,10 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -import os import cudf from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12.py b/gpu_bdb/queries/q12/gpu_bdb_query_12.py index 2bee6465..39d05a42 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12.py @@ -14,10 +14,12 @@ # limitations under the License. # -import sys import os import glob +import cudf +import dask_cudf + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -71,7 +73,6 @@ def filter_wcs_table(web_clickstreams_fn, filtered_item_df): ## AND wcs_user_sk IS NOT NULL ### AND wcs_sales_sk IS NULL --only views, not purchases """ - import cudf web_clickstreams_cols = [ "wcs_user_sk", @@ -131,7 +132,6 @@ def filter_ss_table(store_sales_df, filtered_item_df): def main(client, config): - import cudf, dask_cudf item_df, store_sales_df = benchmark( read_tables, @@ -223,8 +223,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py index 108d1602..2656553a 100755 --- a/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py +++ b/gpu_bdb/queries/q12/gpu_bdb_query_12_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13.py b/gpu_bdb/queries/q13/gpu_bdb_query_13.py index fba18806..79afefe9 100755 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13.py @@ -14,8 +14,6 @@ # limitations under the License. # -import sys - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -191,8 +189,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py index 20177648..19c501f9 100644 --- a/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py +++ b/gpu_bdb/queries/q13/gpu_bdb_query_13_dask_sql.py @@ -14,11 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -from dask.distributed import Client -import os from bdb_tools.utils import ( benchmark, diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14.py b/gpu_bdb/queries/q14/gpu_bdb_query_14.py index 7112f832..2549910b 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14.py @@ -14,11 +14,9 @@ # limitations under the License. # -from dask.distributed import Client - import numpy as np -import sys +import cudf from bdb_tools.utils import ( benchmark, @@ -28,7 +26,6 @@ from bdb_tools.q14_utils import read_tables def main(client, config): - import cudf q14_dependents = 5 q14_morning_startHour = 7 @@ -115,8 +112,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py index 120175e8..ca6850f2 100755 --- a/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py +++ b/gpu_bdb/queries/q14/gpu_bdb_query_14_dask_sql.py @@ -14,10 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -import os from bdb_tools.utils import ( benchmark, diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15.py b/gpu_bdb/queries/q15/gpu_bdb_query_15.py index 3a13fada..e8e24cd4 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15.py @@ -14,10 +14,6 @@ # limitations under the License. # -import sys -from collections import OrderedDict - - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -33,7 +29,6 @@ ) import datetime -import numpy as np def main(client, config): @@ -148,8 +143,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py index d990abbb..c04ea8c2 100755 --- a/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py +++ b/gpu_bdb/queries/q15/gpu_bdb_query_15_dask_sql.py @@ -14,10 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -import os from bdb_tools.utils import ( benchmark, diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16.py b/gpu_bdb/queries/q16/gpu_bdb_query_16.py index a06bc86a..7f2747b4 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16.py @@ -14,7 +14,7 @@ # limitations under the License. # -import sys +import cudf from bdb_tools.utils import ( benchmark, @@ -60,7 +60,6 @@ def get_before_after_sales(df, q16_timestamp): def main(client, config): - import cudf web_sales_df, web_returns_df, date_dim_df, item_df, warehouse_df = benchmark( read_tables, @@ -236,8 +235,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py index 4d9da53a..8ddb145e 100755 --- a/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py +++ b/gpu_bdb/queries/q16/gpu_bdb_query_16_dask_sql.py @@ -14,10 +14,7 @@ # limitations under the License. # -import sys - from bdb_tools.cluster_startup import attach_to_cluster -import os import datetime from datetime import timedelta diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17.py b/gpu_bdb/queries/q17/gpu_bdb_query_17.py index 6d7f860d..c7113e48 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17.py @@ -14,8 +14,7 @@ # limitations under the License. # -import sys -from collections import OrderedDict +import cudf from bdb_tools.utils import ( benchmark, @@ -34,7 +33,6 @@ q17_i_category_IN = "Books", "Music" def main(client, config): - import cudf ( store_sales_df, @@ -169,8 +167,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py index 411879cd..d65181e7 100755 --- a/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py +++ b/gpu_bdb/queries/q17/gpu_bdb_query_17_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q18/gpu_bdb_query_18.py b/gpu_bdb/queries/q18/gpu_bdb_query_18.py index cabe6f2d..f8c260a1 100755 --- a/gpu_bdb/queries/q18/gpu_bdb_query_18.py +++ b/gpu_bdb/queries/q18/gpu_bdb_query_18.py @@ -16,7 +16,8 @@ import os -from collections import OrderedDict +import cudf +import dask_cudf from bdb_tools.utils import ( benchmark, @@ -42,8 +43,6 @@ TEMP_TABLE1 = "TEMP_TABLE1" def main(client, config): - import cudf - import dask_cudf store_sales, date_dim, store, product_reviews = benchmark( read_tables, @@ -125,7 +124,6 @@ def main(client, config): .to_arrow() .to_pylist() ) - n_targets = len(targets) no_nulls = pr[~pr.pr_review_content.isnull()].reset_index(drop=True) no_nulls["pr_review_sk"] = no_nulls["pr_review_sk"].astype("int32") @@ -226,8 +224,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19.py b/gpu_bdb/queries/q19/gpu_bdb_query_19.py index 827c6a7a..a4f0d6f8 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19.py @@ -14,9 +14,11 @@ # limitations under the License. # -import sys import os +import cudf +import dask_cudf + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -28,14 +30,11 @@ read_tables ) -from dask.distributed import Client, wait -import distributed +from dask.distributed import wait q19_returns_dates = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] def main(client, config): - import cudf - import dask_cudf date_dim_df, store_returns_df, web_returns_df, product_reviews_df = benchmark( read_tables, @@ -177,8 +176,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index ba7a66fe..058804b3 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -14,7 +14,6 @@ # limitations under the License. # -import sys import os import dask_cudf diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20.py b/gpu_bdb/queries/q20/gpu_bdb_query_20.py index 8559f101..85c30cfb 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20.py @@ -19,14 +19,12 @@ from bdb_tools.utils import ( benchmark, gpubdb_argparser, - train_clustering_model, run_query, ) from bdb_tools.q20_utils import ( get_clusters, read_tables ) -from dask import delayed from dask.distributed import wait def remove_inf_and_nulls(df, column_names, value=0.0): @@ -164,8 +162,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py index 71f275f9..4715177d 100755 --- a/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py +++ b/gpu_bdb/queries/q20/gpu_bdb_query_20_dask_sql.py @@ -14,13 +14,8 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster -from dask import delayed from dask.distributed import wait -import numpy as np from bdb_tools.utils import ( benchmark, diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21.py b/gpu_bdb/queries/q21/gpu_bdb_query_21.py index 7cb4c8a8..084e0392 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21.py @@ -14,8 +14,6 @@ # limitations under the License. # -import sys - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -25,7 +23,7 @@ from bdb_tools.q21_utils import read_tables -from dask.distributed import Client, wait +from dask.distributed import wait q21_year = 2003 q21_month = 1 @@ -242,8 +240,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py index cb925cd9..eca3dd03 100755 --- a/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py +++ b/gpu_bdb/queries/q21/gpu_bdb_query_21_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22.py b/gpu_bdb/queries/q22/gpu_bdb_query_22.py index 082979a7..e859c660 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22.py @@ -14,10 +14,7 @@ # limitations under the License. # -from numba import cuda import numpy as np -import sys - from bdb_tools.utils import ( benchmark, @@ -134,8 +131,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py index 67834725..6393842c 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22_dask_sql.py @@ -15,8 +15,6 @@ # import numpy as np -import sys -import os from bdb_tools.cluster_startup import attach_to_cluster diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23.py b/gpu_bdb/queries/q23/gpu_bdb_query_23.py index 4e9621c4..0dcb558a 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23.py @@ -14,10 +14,6 @@ # limitations under the License. # -import cupy as cp -import sys -import rmm - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -107,8 +103,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py index b7d9f943..ef3debd8 100755 --- a/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py +++ b/gpu_bdb/queries/q23/gpu_bdb_query_23_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24.py b/gpu_bdb/queries/q24/gpu_bdb_query_24.py index 1aedd1ab..90f2bf3b 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys - - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -226,8 +223,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py index 00c06027..f418beb2 100755 --- a/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py +++ b/gpu_bdb/queries/q24/gpu_bdb_query_24_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25.py b/gpu_bdb/queries/q25/gpu_bdb_query_25.py index e0d64816..2e0fde62 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25.py @@ -14,10 +14,9 @@ # limitations under the License. # -import sys - import numpy as np -from numba import cuda + +import dask_cudf from bdb_tools.utils import ( benchmark, @@ -52,7 +51,6 @@ def agg_count_distinct(df, group_key, counted_key, client): def get_clusters(client, ml_input_df): - import dask_cudf ml_tasks = [ delayed(train_clustering_model)(df, N_CLUSTERS, CLUSTER_ITERATIONS, N_ITER) @@ -75,7 +73,6 @@ def get_clusters(client, ml_input_df): def main(client, config): - import dask_cudf ss_ddf, ws_ddf, datedim_ddf = benchmark( read_tables, @@ -148,8 +145,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py index 85eb9dcf..3ae0afad 100755 --- a/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py +++ b/gpu_bdb/queries/q25/gpu_bdb_query_25_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( @@ -76,7 +73,6 @@ def agg_count_distinct(df, group_key, counted_key): def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) - q25_date = "2002-01-02" ss_join_query= f""" SELECT ss_customer_sk, diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26.py b/gpu_bdb/queries/q26/gpu_bdb_query_26.py index 5169cb74..aae2b999 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26.py @@ -14,11 +14,6 @@ # limitations under the License. # -import sys - -import numpy as np -from numba import cuda - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -121,8 +116,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py index d452c2a9..95458d56 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27.py b/gpu_bdb/queries/q27/gpu_bdb_query_27.py index afa939a8..0634d2a8 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27.py @@ -14,13 +14,11 @@ # limitations under the License. # -import time -import argparse +import dask_cudf from bdb_tools.utils import ( benchmark, gpubdb_argparser, - left_semi_join, run_query ) @@ -39,7 +37,6 @@ from dask.distributed import wait def main(client, config): - import dask_cudf product_reviews_df = benchmark( read_tables, @@ -107,8 +104,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py index 52b20fef..50e27a5a 100755 --- a/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py +++ b/gpu_bdb/queries/q27/gpu_bdb_query_27_dask_sql.py @@ -35,8 +35,6 @@ read_tables ) -from dask.distributed import wait - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) diff --git a/gpu_bdb/queries/q28/gpu_bdb_query_28.py b/gpu_bdb/queries/q28/gpu_bdb_query_28.py index 1ec0ac63..281c84ac 100755 --- a/gpu_bdb/queries/q28/gpu_bdb_query_28.py +++ b/gpu_bdb/queries/q28/gpu_bdb_query_28.py @@ -15,18 +15,9 @@ # import cupy -import dask -import distributed -import numpy as np -import time import cupy as cp import copyreg -import sys, os -import traceback - -from distributed import wait -from cuml.feature_extraction.text import HashingVectorizer from bdb_tools.utils import ( benchmark, @@ -40,7 +31,6 @@ ) def main(client, config): - q_st = time.time() product_reviews_df = benchmark( read_tables, config=config, @@ -81,11 +71,6 @@ def serialize_mat_descriptor(m): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - from cuml.dask.naive_bayes import MultinomialNB as DistMNB - from cuml.dask.common.input_utils import DistributedDataHandler - from cuml.dask.common import to_dask_cudf - config = gpubdb_argparser() client, bc = attach_to_cluster(config) run_query(config=config, client=client, query_func=main) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29.py b/gpu_bdb/queries/q29/gpu_bdb_query_29.py index dbc6fa6b..7dc8c29c 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29.py @@ -14,8 +14,6 @@ # limitations under the License. # -import sys - from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -25,8 +23,6 @@ q29_limit, read_tables ) -from bdb_tools.utils import benchmark -from distributed import wait ### Implementation Notes: # * `drop_duplicates` and `groupby` by default brings result to single partition @@ -135,8 +131,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py index 75f409d4..cb34a5e9 100755 --- a/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py +++ b/gpu_bdb/queries/q29/gpu_bdb_query_29_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30.py b/gpu_bdb/queries/q30/gpu_bdb_query_30.py index af00d031..1f37f718 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30.py @@ -14,10 +14,12 @@ # limitations under the License. # -import sys import glob import os +import cudf +import dask_cudf + from bdb_tools.utils import ( benchmark, gpubdb_argparser, @@ -28,7 +30,7 @@ q30_limit, read_tables ) -from bdb_tools.sessionization import get_session_id, get_distinct_sessions, get_pairs +from bdb_tools.sessionization import get_distinct_sessions, get_pairs from dask import delayed import numpy as np @@ -43,7 +45,6 @@ def pre_repartition_task(wcs_fn, f_item_df): """ Runs the pre-repartition task """ - import cudf wcs_cols = ["wcs_user_sk", "wcs_item_sk", "wcs_click_date_sk", "wcs_click_time_sk"] wcs_df = cudf.read_parquet(wcs_fn, columns=wcs_cols) @@ -65,8 +66,6 @@ def pre_repartition_task(wcs_fn, f_item_df): def main(client, config): - import dask_cudf - import cudf item_df = benchmark( read_tables, @@ -148,8 +147,6 @@ def main(client, config): if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster - import cudf - import dask_cudf config = gpubdb_argparser() client, bc = attach_to_cluster(config) diff --git a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py index 18050907..d7ca3868 100755 --- a/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py +++ b/gpu_bdb/queries/q30/gpu_bdb_query_30_dask_sql.py @@ -14,9 +14,6 @@ # limitations under the License. # -import sys -import os - from bdb_tools.cluster_startup import attach_to_cluster from bdb_tools.utils import ( From a55eddcbb67e7a244f66787bd324700b6da936cb Mon Sep 17 00:00:00 2001 From: sft-managed Date: Thu, 20 Jan 2022 22:13:50 +0000 Subject: [PATCH 39/51] Cleanup remaining repeated code --- gpu_bdb/bdb_tools/q01_utils.py | 2 ++ gpu_bdb/bdb_tools/q03_utils.py | 1 + gpu_bdb/bdb_tools/q09_utils.py | 3 +++ gpu_bdb/queries/q01/gpu_bdb_query_01.py | 4 ++-- gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py | 9 ++++----- gpu_bdb/queries/q03/gpu_bdb_query_03.py | 3 +-- gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py | 5 ++--- gpu_bdb/queries/q04/gpu_bdb_query_04.py | 3 --- gpu_bdb/queries/q09/gpu_bdb_query_09.py | 7 +++---- gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py | 13 ++++++------- gpu_bdb/queries/q19/gpu_bdb_query_19.py | 5 ++--- gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py | 3 +-- 12 files changed, 27 insertions(+), 31 deletions(-) diff --git a/gpu_bdb/bdb_tools/q01_utils.py b/gpu_bdb/bdb_tools/q01_utils.py index 5212f4f3..471b96f8 100644 --- a/gpu_bdb/bdb_tools/q01_utils.py +++ b/gpu_bdb/bdb_tools/q01_utils.py @@ -18,7 +18,9 @@ from bdb_tools.readers import build_reader # -------- Q1 ----------- +q01_i_category_id_IN = 1, 2, 3 # -- sf1 -> 11 stores, 90k sales in 820k lines +q01_ss_store_sk_IN = 10, 20, 33, 40, 50 q01_viewed_together_count = 50 q01_limit = 100 diff --git a/gpu_bdb/bdb_tools/q03_utils.py b/gpu_bdb/bdb_tools/q03_utils.py index 472685ca..1a4fb387 100644 --- a/gpu_bdb/bdb_tools/q03_utils.py +++ b/gpu_bdb/bdb_tools/q03_utils.py @@ -23,6 +23,7 @@ q03_days_in_sec_before_purchase = 864000 q03_views_before_purchase = 5 q03_purchased_item_IN = 10001 +q03_purchased_item_category_IN = 2, 3 q03_limit = 100 def read_tables(config, c=None): diff --git a/gpu_bdb/bdb_tools/q09_utils.py b/gpu_bdb/bdb_tools/q09_utils.py index 8ff86074..42fce78d 100644 --- a/gpu_bdb/bdb_tools/q09_utils.py +++ b/gpu_bdb/bdb_tools/q09_utils.py @@ -21,6 +21,7 @@ q09_year = 2001 q09_part1_ca_country = "United States" +q09_part1_ca_state_IN = "KY", "GA", "NM" q09_part1_net_profit_min = 0 q09_part1_net_profit_max = 2000 q09_part1_education_status = "4 yr Degree" @@ -29,6 +30,7 @@ q09_part1_sales_price_max = 150 q09_part2_ca_country = "United States" +q09_part2_ca_state_IN = "MT", "OR", "IN" q09_part2_net_profit_min = 150 q09_part2_net_profit_max = 3000 q09_part2_education_status = "4 yr Degree" @@ -37,6 +39,7 @@ q09_part2_sales_price_max = 200 q09_part3_ca_country = "United States" +q09_part3_ca_state_IN = "WI", "MO", "WV" q09_part3_net_profit_min = 50 q09_part3_net_profit_max = 25000 q09_part3_education_status = "4 yr Degree" diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01.py b/gpu_bdb/queries/q01/gpu_bdb_query_01.py index f0838ff2..6cd6bbd7 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01.py @@ -17,6 +17,8 @@ from bdb_tools.utils import benchmark, gpubdb_argparser, run_query from bdb_tools.q01_utils import ( + q01_i_category_id_IN, + q01_ss_store_sk_IN, q01_viewed_together_count, q01_limit, read_tables @@ -30,8 +32,6 @@ ### Future Notes: # Settinng index + merge using map_parition can be a work-around if dask native merge is slow -q01_i_category_id_IN = ["1", "2"," 3"] -q01_ss_store_sk_IN = ["10", "20", "33", "40", "50"] ### Inner Self join to get pairs # Select t1.ss_item_sk as item_sk_1 , t2.ss_item_sk as item_sk_2 diff --git a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py index 6cc4428d..9d0f21ad 100755 --- a/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py +++ b/gpu_bdb/queries/q01/gpu_bdb_query_01_dask_sql.py @@ -23,6 +23,8 @@ ) from bdb_tools.q01_utils import ( + q01_i_category_id_IN, + q01_ss_store_sk_IN, q01_viewed_together_count, q01_limit, read_tables @@ -30,9 +32,6 @@ from dask.distributed import wait -q01_i_category_id_IN = "1, 2, 3" -q01_ss_store_sk_IN = ["10", "20", "33", "40", "50"] - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) @@ -40,8 +39,8 @@ def main(data_dir, client, c, config): SELECT DISTINCT ss_item_sk, ss_ticket_number FROM store_sales s, item i WHERE s.ss_item_sk = i.i_item_sk - AND i.i_category_id IN ({q01_i_category_id_IN}) - AND s.ss_store_sk IN ({q01_ss_store_sk_IN}) + AND i.i_category_id IN {q01_i_category_id_IN} + AND s.ss_store_sk IN {q01_ss_store_sk_IN} """ result_distinct = c.sql(query_distinct) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03.py b/gpu_bdb/queries/q03/gpu_bdb_query_03.py index b0e0562f..fd78815f 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03.py @@ -28,6 +28,7 @@ from bdb_tools.q03_utils import ( apply_find_items_viewed, q03_purchased_item_IN, + q03_purchased_item_category_IN, q03_limit, read_tables ) @@ -38,8 +39,6 @@ import glob from dask import delayed -q03_purchased_item_category_IN = [2, 3] - def get_wcs_minima(config): wcs_df = dask_cudf.read_parquet( diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py index e7f200ac..031dfe0f 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03_dask_sql.py @@ -25,6 +25,7 @@ from bdb_tools.q03_utils import ( apply_find_items_viewed, q03_purchased_item_IN, + q03_purchased_item_category_IN, q03_limit, read_tables ) @@ -32,8 +33,6 @@ from dask.distributed import wait -q03_purchased_item_category_IN = "2,3" - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) @@ -64,7 +63,7 @@ def main(data_dir, client, c, config): query_3 = f""" SELECT i_item_sk, i_category_id FROM item_df - WHERE i_category_id IN ({q03_purchased_item_category_IN}) + WHERE i_category_id IN {q03_purchased_item_category_IN} """ item_df_filtered = c.sql(query_3) diff --git a/gpu_bdb/queries/q04/gpu_bdb_query_04.py b/gpu_bdb/queries/q04/gpu_bdb_query_04.py index 0e798bd7..98fba61e 100755 --- a/gpu_bdb/queries/q04/gpu_bdb_query_04.py +++ b/gpu_bdb/queries/q04/gpu_bdb_query_04.py @@ -27,9 +27,6 @@ read_tables ) -# parameters -q04_session_timeout_inSec = 3600 - def main(client, config): wp, wcs_df = benchmark( diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09.py b/gpu_bdb/queries/q09/gpu_bdb_query_09.py index 36a3bf80..0d44b61a 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09.py @@ -25,6 +25,7 @@ from bdb_tools.q09_utils import ( q09_year, q09_part1_ca_country, + q09_part1_ca_state_IN, q09_part1_net_profit_min, q09_part1_net_profit_max, q09_part1_education_status, @@ -32,6 +33,7 @@ q09_part1_sales_price_min, q09_part1_sales_price_max, q09_part2_ca_country, + q09_part2_ca_state_IN, q09_part2_net_profit_min, q09_part2_net_profit_max, q09_part2_education_status, @@ -39,6 +41,7 @@ q09_part2_sales_price_min, q09_part2_sales_price_max, q09_part3_ca_country, + q09_part3_ca_state_IN, q09_part3_net_profit_min, q09_part3_net_profit_max, q09_part3_education_status, @@ -50,10 +53,6 @@ def main(client, config): - q09_part1_ca_state_IN = "KY", "GA", "NM" - q09_part2_ca_state_IN = "MT", "OR", "IN" - q09_part3_ca_state_IN = "WI", "MO", "WV" - ( store_sales, customer_address, diff --git a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py index d13835c1..16e71c7c 100755 --- a/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py +++ b/gpu_bdb/queries/q09/gpu_bdb_query_09_dask_sql.py @@ -25,6 +25,7 @@ from bdb_tools.q09_utils import ( q09_year, q09_part1_ca_country, + q09_part1_ca_state_IN, q09_part1_net_profit_min, q09_part1_net_profit_max, q09_part1_education_status, @@ -32,6 +33,7 @@ q09_part1_sales_price_min, q09_part1_sales_price_max, q09_part2_ca_country, + q09_part2_ca_state_IN, q09_part2_net_profit_min, q09_part2_net_profit_max, q09_part2_education_status, @@ -39,6 +41,7 @@ q09_part2_sales_price_min, q09_part2_sales_price_max, q09_part3_ca_country, + q09_part3_ca_state_IN, q09_part3_net_profit_min, q09_part3_net_profit_max, q09_part3_education_status, @@ -49,10 +52,6 @@ ) -q09_part1_ca_state_IN = "'KY', 'GA', 'NM'" -q09_part2_ca_state_IN = "'MT', 'OR', 'IN'" -q09_part3_ca_state_IN = "'WI', 'MO', 'WV'" - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) @@ -95,21 +94,21 @@ def main(data_dir, client, c, config): ( ( ca1.ca_country = '{q09_part1_ca_country}' - AND ca1.ca_state IN ({q09_part1_ca_state_IN}) + AND ca1.ca_state IN {q09_part1_ca_state_IN} AND {q09_part1_net_profit_min} <= ss1.ss_net_profit AND ss1.ss_net_profit <= {q09_part1_net_profit_max} ) OR ( ca1.ca_country = '{q09_part2_ca_country}' - AND ca1.ca_state IN ({q09_part2_ca_state_IN}) + AND ca1.ca_state IN {q09_part2_ca_state_IN} AND {q09_part2_net_profit_min} <= ss1.ss_net_profit AND ss1.ss_net_profit <= {q09_part2_net_profit_max} ) OR ( ca1.ca_country = '{q09_part3_ca_country}' - AND ca1.ca_state IN ({q09_part3_ca_state_IN}) + AND ca1.ca_state IN {q09_part3_ca_state_IN} AND {q09_part3_net_profit_min} <= ss1.ss_net_profit AND ss1.ss_net_profit <= {q09_part3_net_profit_max} ) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19.py b/gpu_bdb/queries/q19/gpu_bdb_query_19.py index a4f0d6f8..06b51580 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19.py @@ -26,14 +26,13 @@ ) from bdb_tools.text import create_sentences_from_reviews, create_words_from_sentences from bdb_tools.q19_utils import ( + q19_returns_dates_IN, eol_char, read_tables ) from dask.distributed import wait -q19_returns_dates = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] - def main(client, config): date_dim_df, store_returns_df, web_returns_df, product_reviews_df = benchmark( @@ -47,7 +46,7 @@ def main(client, config): date_dim_df = date_dim_df.merge( date_dim_df, on=["d_week_seq"], how="outer", suffixes=("", "_r") ) - date_dim_df = date_dim_df[date_dim_df.d_date_r.isin(q19_returns_dates)].reset_index( + date_dim_df = date_dim_df[date_dim_df.d_date_r.isin(q19_returns_dates_IN)].reset_index( drop=True ) diff --git a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py index 058804b3..fc2b6183 100755 --- a/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py +++ b/gpu_bdb/queries/q19/gpu_bdb_query_19_dask_sql.py @@ -32,14 +32,13 @@ ) from bdb_tools.q19_utils import ( + q19_returns_dates_IN, eol_char, read_tables ) from dask.distributed import wait -q19_returns_dates_IN = ["2004-03-08", "2004-08-02", "2004-11-15", "2004-12-20"] - def main(data_dir, client, c, config): benchmark(read_tables, config, c, dask_profile=config["dask_profile"]) From d73b1c04d9b50fadc8be316d185f67b34ee16cca Mon Sep 17 00:00:00 2001 From: sft-managed Date: Tue, 25 Jan 2022 18:01:44 +0000 Subject: [PATCH 40/51] Add dask-sql environment file --- conda/rapids-gpu-bdb-dask-sql.yml | 34 +++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100755 conda/rapids-gpu-bdb-dask-sql.yml diff --git a/conda/rapids-gpu-bdb-dask-sql.yml b/conda/rapids-gpu-bdb-dask-sql.yml new file mode 100755 index 00000000..dfc9687a --- /dev/null +++ b/conda/rapids-gpu-bdb-dask-sql.yml @@ -0,0 +1,34 @@ +channels: + - rapidsai-nightly + - nvidia + - conda-forge + +dependencies: + - python=3.8 + - cudatoolkit=11.2 + - cudf + - rmm + - dask-cuda + - dask-cudf + - cuml + - dask + - distributed + - ucx-py + - ucx-proc=*=gpu + - dask-sql + - numba=0.54.* + - scipy + - scikit-learn + - cupy + - spacy=2.3 + - oauth2client + - asyncssh + - psutil + - ipykernel + - jupyterlab + - gspread + - oauth2client + - pytest + - pip + - pip: + - jupyter-server-proxy From a9a0833b6c2ad26fed83f3cdf2df34084b8668a5 Mon Sep 17 00:00:00 2001 From: sft-managed Date: Tue, 25 Jan 2022 20:43:29 +0000 Subject: [PATCH 41/51] Update dask-sql version --- conda/rapids-gpu-bdb-dask-sql.yml | 2 +- gpu_bdb/benchmark_runner.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/conda/rapids-gpu-bdb-dask-sql.yml b/conda/rapids-gpu-bdb-dask-sql.yml index dfc9687a..54156f8c 100755 --- a/conda/rapids-gpu-bdb-dask-sql.yml +++ b/conda/rapids-gpu-bdb-dask-sql.yml @@ -15,7 +15,7 @@ dependencies: - distributed - ucx-py - ucx-proc=*=gpu - - dask-sql + - dask-sql>=2022.1 - numba=0.54.* - scipy - scikit-learn diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index 4c6a3a8b..fe055016 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -56,7 +56,7 @@ def load_query(qnum, fn): print("Dask SQL Queries") for r in range(N_REPEATS): for qnum, q_func in sql_queries.items(): - print(f"{r}: {qnum}") + print(f"run {r+1}: q{qnum}") qpath = f"{base_path}/queries/q{qnum}/" os.chdir(qpath) @@ -81,7 +81,7 @@ def load_query(qnum, fn): print("Pure Dask Queries") for r in range(N_REPEATS): for qnum, q_func in dask_queries.items(): - print(f"{r}: {qnum}") + print(f"run {r+1}: q{qnum}") qpath = f"{base_path}/queries/q{qnum}/" os.chdir(qpath) From b60d2f19e0c9b5e5a0e4962900fe695767b42bf1 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Tue, 25 Jan 2022 13:46:22 -0800 Subject: [PATCH 42/51] fix query 22 --- gpu_bdb/queries/q22/gpu_bdb_query_22.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22.py b/gpu_bdb/queries/q22/gpu_bdb_query_22.py index e859c660..2b1ecb80 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22.py @@ -49,8 +49,8 @@ def main(client, config): "i_current_price >= @q22_i_current_price_min and i_current_price<= @q22_i_current_price_max", meta=item._meta, local_dict={ - "q22_i_current_price_min": q22_i_current_price_min, - "q22_i_current_price_max": q22_i_current_price_max, + "q22_i_current_price_min": float(q22_i_current_price_min), + "q22_i_current_price_max": float(q22_i_current_price_max), }, ).reset_index(drop=True) From 3cf17b595eaba32da97a67a9e96ead4b9e7dfa19 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Tue, 25 Jan 2022 14:46:35 -0800 Subject: [PATCH 43/51] Query 03 fi --- gpu_bdb/queries/q03/gpu_bdb_query_03.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03.py b/gpu_bdb/queries/q03/gpu_bdb_query_03.py index fd78815f..a2feecc5 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03.py @@ -62,7 +62,7 @@ def pre_repartition_task(wcs_fn, item_df, wcs_tstamp_min): "wcs_click_time_sk", ] wcs_df = cudf.read_parquet(wcs_fn, columns=wcs_cols) - wcs_df = wcs_df._drop_na_rows(subset=["wcs_user_sk", "wcs_item_sk"]) + wcs_df = wcs_df.dropna(axis=0,subset=["wcs_user_sk", "wcs_item_sk"]) wcs_df["tstamp"] = wcs_df["wcs_click_date_sk"] * 86400 + wcs_df["wcs_click_time_sk"] wcs_df["tstamp"] = wcs_df["tstamp"] - wcs_tstamp_min From e3a94c199b7cc4bd590706682f410cbf718d6a2a Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Tue, 25 Jan 2022 14:51:27 -0800 Subject: [PATCH 44/51] small style fix --- gpu_bdb/queries/q03/gpu_bdb_query_03.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gpu_bdb/queries/q03/gpu_bdb_query_03.py b/gpu_bdb/queries/q03/gpu_bdb_query_03.py index a2feecc5..a563ff83 100755 --- a/gpu_bdb/queries/q03/gpu_bdb_query_03.py +++ b/gpu_bdb/queries/q03/gpu_bdb_query_03.py @@ -62,7 +62,7 @@ def pre_repartition_task(wcs_fn, item_df, wcs_tstamp_min): "wcs_click_time_sk", ] wcs_df = cudf.read_parquet(wcs_fn, columns=wcs_cols) - wcs_df = wcs_df.dropna(axis=0,subset=["wcs_user_sk", "wcs_item_sk"]) + wcs_df = wcs_df.dropna(axis=0, subset=["wcs_user_sk", "wcs_item_sk"]) wcs_df["tstamp"] = wcs_df["wcs_click_date_sk"] * 86400 + wcs_df["wcs_click_time_sk"] wcs_df["tstamp"] = wcs_df["tstamp"] - wcs_tstamp_min From ca49940c3fa092f0486fed32b8c44847aa6ffcc1 Mon Sep 17 00:00:00 2001 From: Ayush Dattagupta Date: Wed, 26 Jan 2022 10:07:55 -0800 Subject: [PATCH 45/51] replace deprecated df.one_hot_encoder with cudf.get_dummies --- gpu_bdb/queries/q05/gpu_bdb_query_05.py | 6 +++--- gpu_bdb/queries/q26/gpu_bdb_query_26.py | 7 ++++--- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/gpu_bdb/queries/q05/gpu_bdb_query_05.py b/gpu_bdb/queries/q05/gpu_bdb_query_05.py index ce94f09d..5e99a10f 100755 --- a/gpu_bdb/queries/q05/gpu_bdb_query_05.py +++ b/gpu_bdb/queries/q05/gpu_bdb_query_05.py @@ -57,12 +57,12 @@ def get_groupby_results(file_list, item_df): keep_cols = ["wcs_user_sk", "i_category_id", "clicks_in_category"] wcs_ddf = wcs_ddf[keep_cols] - wcs_ddf = cudf.DataFrame.one_hot_encoding( + wcs_ddf = cudf.get_dummies( wcs_ddf, - column="i_category_id", + columns=["i_category_id"], prefix="clicks_in", prefix_sep="_", - cats=[i for i in range(1, 8)], + cats={"i_category_id":np.arange(1, 8, dtype="int32")}, dtype=np.int8, ) keep_cols = ["wcs_user_sk", "clicks_in_category"] + [ diff --git a/gpu_bdb/queries/q26/gpu_bdb_query_26.py b/gpu_bdb/queries/q26/gpu_bdb_query_26.py index aae2b999..fa4b81b3 100755 --- a/gpu_bdb/queries/q26/gpu_bdb_query_26.py +++ b/gpu_bdb/queries/q26/gpu_bdb_query_26.py @@ -28,6 +28,7 @@ N_ITER, read_tables ) +import numpy as np from dask import delayed def agg_count_distinct(df, group_key, counted_key): @@ -90,10 +91,10 @@ def main(client, config): # One-Hot-Encode i_class_id merged_ddf = merged_ddf.map_partitions( - cudf.DataFrame.one_hot_encoding, - column="i_class_id", + cudf.get_dummies, + columns=["i_class_id"], prefix="id", - cats=[i for i in range(1, 16)], + cats={"i_class_id": np.arange(1, 16, dtype="int32")}, prefix_sep="", dtype="float32", ) From 1dcc3f5e94ca8d8ff4f633d90b45371e6f01db23 Mon Sep 17 00:00:00 2001 From: Vibhu Jawa Date: Wed, 26 Jan 2022 11:30:00 -0800 Subject: [PATCH 46/51] Q22 result verfied --- gpu_bdb/bdb_tools/q22_utils.py | 4 ++-- gpu_bdb/queries/q22/gpu_bdb_query_22.py | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/gpu_bdb/bdb_tools/q22_utils.py b/gpu_bdb/bdb_tools/q22_utils.py index b48ed504..db44f325 100644 --- a/gpu_bdb/bdb_tools/q22_utils.py +++ b/gpu_bdb/bdb_tools/q22_utils.py @@ -18,8 +18,8 @@ from bdb_tools.utils import convert_datestring_to_days q22_date = "2001-05-08" -q22_i_current_price_min = "0.98" -q22_i_current_price_max = "1.5" +q22_i_current_price_min = 0.98 +q22_i_current_price_max = 1.5 def read_tables(config, c=None): diff --git a/gpu_bdb/queries/q22/gpu_bdb_query_22.py b/gpu_bdb/queries/q22/gpu_bdb_query_22.py index 2b1ecb80..3a56d3fb 100755 --- a/gpu_bdb/queries/q22/gpu_bdb_query_22.py +++ b/gpu_bdb/queries/q22/gpu_bdb_query_22.py @@ -20,7 +20,6 @@ benchmark, gpubdb_argparser, run_query, - convert_datestring_to_days, ) from bdb_tools.q22_utils import ( q22_date, @@ -49,8 +48,8 @@ def main(client, config): "i_current_price >= @q22_i_current_price_min and i_current_price<= @q22_i_current_price_max", meta=item._meta, local_dict={ - "q22_i_current_price_min": float(q22_i_current_price_min), - "q22_i_current_price_max": float(q22_i_current_price_max), + "q22_i_current_price_min": q22_i_current_price_min, + "q22_i_current_price_max": q22_i_current_price_max, }, ).reset_index(drop=True) @@ -69,7 +68,6 @@ def main(client, config): output_table = output_table[keep_columns] - date_dim = date_dim.map_partitions(convert_datestring_to_days) # Filter limit in days min_date = np.datetime64(q22_date, "D").astype(int) - 30 From 99e5f30e1557b50dec2bc444a0c649dc5b46749c Mon Sep 17 00:00:00 2001 From: Kevin German Date: Mon, 20 Dec 2021 08:06:55 -0800 Subject: [PATCH 47/51] log dask task stream and rmm events into results dir --- gpu_bdb/bdb_tools/__init__.py | 3 ++ gpu_bdb/bdb_tools/dasktasklogger.py | 22 +++++++++ gpu_bdb/bdb_tools/rmm_monitor.py | 76 +++++++++++++++++++++++++++++ 3 files changed, 101 insertions(+) create mode 100644 gpu_bdb/bdb_tools/dasktasklogger.py create mode 100644 gpu_bdb/bdb_tools/rmm_monitor.py diff --git a/gpu_bdb/bdb_tools/__init__.py b/gpu_bdb/bdb_tools/__init__.py index ccbb1625..2b586df8 100755 --- a/gpu_bdb/bdb_tools/__init__.py +++ b/gpu_bdb/bdb_tools/__init__.py @@ -1 +1,4 @@ # Copyright (c) 2020, NVIDIA CORPORATION. + +from .rmm_monitor import RMMResourceMonitor +from .dasktasklogger import DaskTaskLogger diff --git a/gpu_bdb/bdb_tools/dasktasklogger.py b/gpu_bdb/bdb_tools/dasktasklogger.py new file mode 100644 index 00000000..38532ffc --- /dev/null +++ b/gpu_bdb/bdb_tools/dasktasklogger.py @@ -0,0 +1,22 @@ +import re +import os +import json +import numpy as np + +example={'worker': 'tcp://10.180.4.206:42643', 'status': 'OK', 'nbytes': 28263552, 'thread': 140534840973056, 'type': b'\x80\x04\x95%\x00\x00\x00\x00\x00\x00\x00\x8c\x13cudf.core.dataframe\x94\x8c\tDataFrame\x94\x93\x94.', 'typename': 'cudf.core.dataframe.DataFrame', 'metadata': {}, 'startstops': ({'action': 'transfer', 'start': 1639787413.9825313, 'stop': 1639787413.998216, 'source': 'tcp://10.180.4.206:45115'}, {'action': 'compute', 'start': 1639787413.998873, 'stop': 1639787414.0106611}), 'key': "('drop-duplicates-combine-d121e7e64a9ef70e5616e411e95f2d3e', 1, 8, 0)"} + +class DaskTaskLogger(): + key_expr=re.compile( '([\w-]+)-([0-9a-f-]{32,36})' ) + + def __init__(self, client, outputdir='/tmp'): + self._client=client + self._outputdir=outputdir + + def mark_begin( self ): + self._client.get_task_stream() + + def save_tasks( self, prefix='dask' ): + plotfname=os.path.join(self._outputdir, f"{prefix}_plot.html") + pdata, pfigure = self._client.get_task_stream(plot='save', filename=plotfname) + with open( os.path.join(self._outputdir, f"{prefix}_tasks.json"), 'w') as outf: + json.dump([{k:t[k] for k in filter( lambda x: type(t[x]) != bytes().__class__, t)} for t in pdata],outf) diff --git a/gpu_bdb/bdb_tools/rmm_monitor.py b/gpu_bdb/bdb_tools/rmm_monitor.py new file mode 100644 index 00000000..b6bca333 --- /dev/null +++ b/gpu_bdb/bdb_tools/rmm_monitor.py @@ -0,0 +1,76 @@ +import os +import csv +import rmm +import tempfile +import asyncio + +from dask.distributed import Client, Worker, WorkerPlugin + +from typing import List + + +class DependencyInstaller(WorkerPlugin): + def __init__(self, dependencies: List[str]): + self._depencendies = " ".join(f"'{dep}'" for dep in dependencies) + + def setup(self, _worker: Worker): + os.system(f"conda install -c rapidsai-nightly -c rapidsai -c nvidia -c conda-forge -c defaults {self._depencendies}") + +# Wrap this in a method used to initialize the module + pass in teh client instance +dependency_installer = DependencyInstaller(["pynvml"]) + +#client = Client() +#client.register_worker_plugin(dependency_installer) + +class RMMResourceMonitor: + """ + Distributed montor for RMM resource allocations + """ + + def __init__( self, client, outputdir='/tmp' ): + self._client = client if isinstance(client, Client) else None + self._outputdir=outputdir + + def __dispatch__( self, method, **kwargs ): + if self._client: + self._client.run( method, **kwargs ) + else: + return method(*args, **kwargs ) + + def get_remote_output_dir( self ): + return self._outputdir + + def begin_logging( self, prefix="rmmlog"): + """ + enable rmm logging into dask temporary directory + """ + + def _rmmlogstart( basedir, prefix ): + import os + rmm.enable_logging( log_file_name=os.path.join( basedir, f"{prefix}_" + str(os.getpid())+".log")) + + self.__dispatch__( _rmmlogstart, prefix=prefix, basedir=self.get_remote_output_dir()) + + def stop_logging( self ): + """ + disable rmm logging and mark files for retrieval + """ + def _rmmlogstop(): + rmm.disable_logging() + + self.__dispatch__( _rmmlogstop ) + + def collect( self ): + """ + distributed command retrieves an logfile + @return reference to dataframe into which rresults are being loaded + """ + def _collect(): + for fname in (rmm.get_log_filenames()): + print( fname ) + #load into memory and return dask_dataframe reference? + + retval = DaskDataframe() + for lf_future in self.__dispatch__( _collect, localfile ): + pass + From cbd632ecc21b84f885c5cc843e2dc04d978fe6a8 Mon Sep 17 00:00:00 2001 From: Kevin German Date: Thu, 6 Jan 2022 08:52:55 -0800 Subject: [PATCH 48/51] unneccesary --- gpu_bdb/bdb_tools/dasktasklogger.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/gpu_bdb/bdb_tools/dasktasklogger.py b/gpu_bdb/bdb_tools/dasktasklogger.py index 38532ffc..fcb4b9bb 100644 --- a/gpu_bdb/bdb_tools/dasktasklogger.py +++ b/gpu_bdb/bdb_tools/dasktasklogger.py @@ -3,8 +3,6 @@ import json import numpy as np -example={'worker': 'tcp://10.180.4.206:42643', 'status': 'OK', 'nbytes': 28263552, 'thread': 140534840973056, 'type': b'\x80\x04\x95%\x00\x00\x00\x00\x00\x00\x00\x8c\x13cudf.core.dataframe\x94\x8c\tDataFrame\x94\x93\x94.', 'typename': 'cudf.core.dataframe.DataFrame', 'metadata': {}, 'startstops': ({'action': 'transfer', 'start': 1639787413.9825313, 'stop': 1639787413.998216, 'source': 'tcp://10.180.4.206:45115'}, {'action': 'compute', 'start': 1639787413.998873, 'stop': 1639787414.0106611}), 'key': "('drop-duplicates-combine-d121e7e64a9ef70e5616e411e95f2d3e', 1, 8, 0)"} - class DaskTaskLogger(): key_expr=re.compile( '([\w-]+)-([0-9a-f-]{32,36})' ) From d38e827a8098d431e0d432fa24d7aa3735b14dd0 Mon Sep 17 00:00:00 2001 From: Kevin German Date: Thu, 13 Jan 2022 10:17:56 -0800 Subject: [PATCH 49/51] rmm logs named for each worker by pid --- gpu_bdb/bdb_tools/rmm_monitor.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/gpu_bdb/bdb_tools/rmm_monitor.py b/gpu_bdb/bdb_tools/rmm_monitor.py index b6bca333..62e74473 100644 --- a/gpu_bdb/bdb_tools/rmm_monitor.py +++ b/gpu_bdb/bdb_tools/rmm_monitor.py @@ -47,7 +47,9 @@ def begin_logging( self, prefix="rmmlog"): def _rmmlogstart( basedir, prefix ): import os - rmm.enable_logging( log_file_name=os.path.join( basedir, f"{prefix}_" + str(os.getpid())+".log")) + fname=f"{prefix}_{os.getpid()}.log" + rmm.enable_logging( log_file_name=os.path.join( basedir, fname)) + return fname self.__dispatch__( _rmmlogstart, prefix=prefix, basedir=self.get_remote_output_dir()) From db13a589292dab4c67ab5d173215890b04227455 Mon Sep 17 00:00:00 2001 From: Kevin German Date: Tue, 25 Jan 2022 08:38:16 -0800 Subject: [PATCH 50/51] gate logging with config options --- gpu_bdb/benchmark_runner/benchmark_config.yaml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/gpu_bdb/benchmark_runner/benchmark_config.yaml b/gpu_bdb/benchmark_runner/benchmark_config.yaml index db1fc4d6..95af4169 100755 --- a/gpu_bdb/benchmark_runner/benchmark_config.yaml +++ b/gpu_bdb/benchmark_runner/benchmark_config.yaml @@ -1,19 +1,21 @@ # benchmark config yaml ### Please fill these accordingly -data_dir: +data_dir: /raid/gpu-bdb/sf1000/parquet_2gb output_dir: file_format: parquet output_filetype: parquet split_row_groups: False repartition_small_table: True benchmark_runner_include_sql: +benchmark_runner_log_rmm: False +benchmark_runner_log_tasks: False -scheduler_file_path: +scheduler_file_path: /raid/adattagupta/dask-sql-work/dask-local-directory/scheduler.json dask_profile: False verify_results: False verify_dir: -sheet: -tab: +sheet: GPU-BDB Dask-SQL +tab: SF1K Dask-SQL get_read_time: False From e4d78171a6d52cc5f18431982b67fbc4f2d4c542 Mon Sep 17 00:00:00 2001 From: Kevin German Date: Wed, 2 Feb 2022 16:01:33 -0800 Subject: [PATCH 51/51] rebase dask-sql --- gpu_bdb/benchmark_runner.py | 34 ++++++++++++++++++++++++++++++++-- 1 file changed, 32 insertions(+), 2 deletions(-) diff --git a/gpu_bdb/benchmark_runner.py b/gpu_bdb/benchmark_runner.py index fe055016..6c383ab5 100755 --- a/gpu_bdb/benchmark_runner.py +++ b/gpu_bdb/benchmark_runner.py @@ -21,9 +21,11 @@ def load_query(qnum, fn): return mod.main -dask_qnums = [str(i).zfill(2) for i in range(1, 31)] -sql_qnums = [str(i).zfill(2) for i in range(1, 31)] +dask_qnums = [str(i).zfill(2) for i in map(int,os.getenv("DASK_QNUMS"," ".join(map(str,range(1, 31)))).split())] +sql_qnums = [str(i).zfill(2) for i in map(int,os.getenv("BSQL_QNUMS"," ".join(map(str,range(1, 31)))).split())] +from random import shuffle +shuffle(dask_qnums) if __name__ == "__main__": from bdb_tools.cluster_startup import attach_to_cluster, import_query_libs @@ -32,6 +34,7 @@ def load_query(qnum, fn): import_query_libs() config = gpubdb_argparser() config["run_id"] = uuid.uuid4().hex + include_sql = config.get("benchmark_runner_include_sql") dask_queries = { @@ -44,6 +47,11 @@ def load_query(qnum, fn): qnum: load_query(qnum, f"queries/q{qnum}/gpu_bdb_query_{qnum}_dask_sql.py") for qnum in sql_qnums } + else: + dask_queries = { + qnum: load_query(qnum, f"queries/q{qnum}/gpu_bdb_query_{qnum}.py") + for qnum in dask_qnums + } client, c = attach_to_cluster(config, create_sql_context=include_sql) # Preload required libraries for queries on all workers @@ -51,6 +59,26 @@ def load_query(qnum, fn): base_path = os.getcwd() + if config.get('benchmark_runner_log_rmm', False) or config.get('benchmark_runner_log_tasks', False): + + from bdb_tools import RMMResourceMonitor + from bdb_tools import DaskTaskLogger + + rmm_analyzer=RMMResourceMonitor(client=client, + outputdir=os.getenv('OUTPUT_DIR', '/tmp')) + dasktasklog=DaskTaskLogger( client=client, + outputdir=os.getenv('OUTPUT_DIR', '/tmp')) + + orig_run_query=run_query + def logged_run_query( *args, **kwargs ): + rmm_analyzer.begin_logging( prefix=f"rmmlog{qnum}") + dasktasklog.mark_begin() + orig_run_query( *args, **kwargs ) + rmm_analyzer.stop_logging() + dasktasklog.save_tasks( prefix=f"dasktasklog{qnum}") + + run_query=logged_run_query + # Run Dask SQL Queries if include_sql and len(sql_qnums) > 0: print("Dask SQL Queries") @@ -95,3 +123,5 @@ def load_query(qnum, fn): client.run_on_scheduler(gc.collect) gc.collect() time.sleep(3) + +