-
Notifications
You must be signed in to change notification settings - Fork 28
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Initial support for Data Maintenance #9
Changes from 3 commits
3d088db
c4ac29c
8f3714f
bfc2079
a4cbec6
920d84e
fa910a4
cc20271
1c6223c
96bac9d
44c4d39
e405cf1
d072c33
9c3515e
36876fc
9c37ed8
6e82a79
b2d4f0e
220e3b4
b333af9
80969ee
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,5 +21,13 @@ export SPARK_CONF=("--master" "yarn" | |
"--num-executors" "8" | ||
"--executor-memory" "40G" | ||
"--executor-cores" "12" | ||
"--conf" "spark.task.cpus=1") | ||
"--conf" "spark.task.cpus=1" | ||
"--packages" "org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.13.1" | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shall we consider the case of not using iceberg? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @jlowe What's our plan/strategy to do the WHOLE NDS test? Are we going to do them all based on Iceberg? For #8 I've made the transcode step to save the data ONLY to Iceberg, should we keep our old way to save them just to a folder? The old way may be more friendly to users who doesn't know Iceberg, but eventually, if they want to perform the whole NDS test including Data Maintenance, they will come back to do Iceberg writing again... Any suggestions for this? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To do the entire NDS suite one would need to use a format that supports the entire set of operations required by the entire suite. That would mean using Iceberg, Delta Lake, or some other format that allows incremental table update.
That's not desired. We want to support transcoding to a bunch of different formats, because we're not always going to run the entire suite. We get a lot of useful information from running the significant portion of NDS that works on raw Parquet and ORC files, and we do not want to lose the ability to setup those benchmarks. The transcode needs to be flexible, allowing outputs ideally to every major output format that we want to bench. For now that definitely includes raw Parquet and ORC along with Iceberg (and the ability to control settings for these formats such as compression codec, probably via separate configs spec'd either inline or sideband in the Spark instance to use). |
||
"--conf" "spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions" | ||
"--conf" "spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog" | ||
"--conf" "spark.sql.catalog.spark_catalog.type=hadoop" | ||
"--conf" "spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog" | ||
"--conf" "spark.sql.catalog.local.type=hadoop" | ||
"--conf" "spark.sql.catalog.local.warehouse=$PWD/local-warehouse" | ||
"--conf" "spark.sql.catalog.spark_catalog.warehouse=$PWD/spark-warehouse") | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,102 @@ | ||
import argparse | ||
import csv | ||
import time | ||
|
||
from pyspark.sql import SparkSession | ||
from PysparkBenchReport import PysparkBenchReport | ||
|
||
from check import get_abs_path | ||
|
||
def get_maintenance_queries(folder, spec_queries): | ||
"""get query content from DM query files | ||
|
||
Args: | ||
folder (str): folder to Data Maintenance query files | ||
spec_queries (list[str]): specific target Data Maintenance queries | ||
Returns: | ||
dict{str: list[str]}: a dict contains Data Maintenance query name and its content. | ||
""" | ||
#TODO: Add delete functions | ||
DM_FUNCS = ['LF_CR', | ||
'LF_CS', | ||
'LF_I', | ||
'LF_SR', | ||
'LF_SS', | ||
'LF_WR', | ||
'LF_WS'] | ||
if spec_queries: | ||
for q in spec_queries: | ||
if q not in DM_FUNCS: | ||
raise Exception(f"invalid Data Maintenance query: {q}. Valid are: {DM_FUNCS}") | ||
DM_FUNCS = [q for q in spec_queries if q in DM_FUNCS] | ||
folder_abs_path = get_abs_path(folder) | ||
q_dict = {} | ||
for q in DM_FUNCS: | ||
with open(folder_abs_path + '/' + q + '.sql', 'r') as f: | ||
# file content e.g. | ||
# " CREATE view ..... ; INSERT into .... ;" | ||
q_content = [ c + ';' for c in f.read().split(';')[:-1]] | ||
q_dict[q] = q_content | ||
return q_dict | ||
|
||
def run_query(query_dict, time_log_output_path): | ||
# TODO: Duplicate code in nds_power.py. Refactor this part, make it general. | ||
execution_time_list = [] | ||
total_time_start = time.time() | ||
if len(query_dict) == 1: | ||
app_name = "NDS - Data Maintenance - " + list(query_dict.keys())[0] | ||
else: | ||
app_name = "NDS - Data Maintenance" | ||
|
||
spark_session = SparkSession.builder.appName( | ||
app_name).getOrCreate() | ||
spark_app_id = spark_session.sparkContext.applicationId | ||
DM_start = time.time() | ||
for query_name, q_content in query_dict.items(): | ||
# show query name in Spark web UI | ||
spark_session.sparkContext.setJobGroup(query_name, query_name) | ||
print(f"====== Run {query_name} ======") | ||
q_report = PysparkBenchReport(spark_session) | ||
for q in q_content: | ||
summary = q_report.report_on(spark_session.sql, | ||
q) | ||
print(f"Time taken: {summary['queryTimes']} millis for {query_name}") | ||
execution_time_list.append((spark_app_id, query_name, summary['queryTimes'])) | ||
q_report.write_summary(query_name, prefix="") | ||
spark_session.sparkContext.stop() | ||
DM_end = time.time() | ||
DM_elapse = DM_end - DM_start | ||
total_elapse = DM_end - total_time_start | ||
print("====== Data Maintenance Time: {} s ======".format(DM_elapse)) | ||
print("====== Total Time: {} s ======".format(total_elapse)) | ||
execution_time_list.append( | ||
(spark_app_id, "Data Maintenance Time", DM_elapse)) | ||
execution_time_list.append( | ||
(spark_app_id, "Total Time", total_elapse)) | ||
|
||
# write to local csv file | ||
header = ["application_id", "query", "time/s"] | ||
with open(time_log_output_path, 'w', encoding='UTF8') as f: | ||
writer = csv.writer(f) | ||
writer.writerow(header) | ||
writer.writerows(execution_time_list) | ||
|
||
|
||
if __name__ == "__main__": | ||
parser = parser = argparse.ArgumentParser() | ||
parser.add_argument('maintenance_queries_folder', | ||
help='folder contains all NDS Data Maintenance queries. If ' + | ||
'"--maintenance_queries" is not set, all queries under the folder will be' + | ||
'executed.') | ||
parser.add_argument('time_log', | ||
help='path to execution time log, only support local path.', | ||
default="") | ||
parser.add_argument('--maintenance_queries', | ||
type=lambda s: s.split(','), | ||
help='specify Data Maintenance query names by a comma seprated string.' + | ||
' e.g. "LF_CR,LF_CS"') | ||
|
||
args = parser.parse_args() | ||
query_dict = get_maintenance_queries(args.maintenance_queries_folder, | ||
args.maintenance_queries) | ||
run_query(query_dict, args.time_log) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: the first character seems not aligned.