IMPALA-12601: Add a fully partitioned TPC-DS database

The current tpcds dataset only has store_sales table fully partitioned
and leaves the other facts table unpartitioned. This is intended for
faster data loading during tests. However, this is not an accurate
reflection of the larger scale TPC-DS dataset where all facts tables are
partitioned. Impala planner may change the details of the query plan if
a partition column exists.

This patch adds a new dataset tpcds_partitioned, loading a fully
partitioned TPC-DS db in parquet format named
tpcds_partitioned_parquet_snap. This dataset can not be loaded
independently and requires the base 'tpcds' db from the tpcds dataset to
be preloaded first. An example of how to load this dataset can be seen
at function load-tpcds-data in bin/create-load-data.sh.

This patch also changes PlannerTest#testProcessingCost from targeting
tpcds_parquet to tpcds_partitioned_parquet_snap. Other planner tests are
that currently target tpcds_parquet will be gradually changed to test
against tpcds_partitioned_parquet_snap in follow-up patches.

This addition adds a couple of seconds in the "Computing table stats"
step, but loading itself is negligible since it is parallelized with
TPC-H and functional-query. The total loading time for the three
datasets remains similar after this patch.

This patch also adds several improvements in the following files:

bin/load-data.py:
- Log elapsed time on serial steps.

testdata/bin/create-load-data.sh:
- Rename MSG to LOAD_MSG to avoid collision with the same variable name
  in ./testdata/bin/run-step.sh

testdata/bin/generate-schema-statements.py:
- Remove redundant FILE_FORMAT_MAP.
- Add build_partitioned_load to simplify expressing partitioned insert
  query in SQL template.

testdata/datasets/tpcds/tpcds_schema_template.sql:
- Reorder schema template to load all dimension tables before fact tables.

Testing:
- Pass core tests.

Change-Id: I3a2e66c405639554f325ae78c66628d464f6c453
Reviewed-on: http://gerrit.cloudera.org:8080/20756
Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This commit is contained in:
Riza Suminto
2022-12-06 15:34:49 -08:00
committed by Impala Public Jenkins
parent 1141a6a80c
commit 8661f922d3
14 changed files with 2857 additions and 1987 deletions

View File

@@ -34,7 +34,7 @@ import time
import traceback
from optparse import OptionParser
from tests.beeswax.impala_beeswax import *
from tests.beeswax.impala_beeswax import ImpalaBeeswaxClient
from multiprocessing.pool import ThreadPool
LOG = logging.getLogger('load-data.py')
@@ -159,11 +159,17 @@ def exec_hive_query_from_file_beeline(file_name):
return is_success
def exec_hbase_query_from_file(file_name):
def exec_hbase_query_from_file(file_name, step_name):
if not os.path.exists(file_name): return
LOG.info('Begin step "%s".' % step_name)
start_time = time.time()
hbase_cmd = "hbase shell %s" % file_name
LOG.info('Executing HBase Command: %s' % hbase_cmd)
exec_cmd(hbase_cmd, error_msg='Error executing hbase create commands')
total_time = time.time() - start_time
LOG.info('End step "%s". Total time: %.2fs\n' % (step_name, total_time))
# KERBEROS TODO: fails when kerberized and impalad principal isn't "impala"
def exec_impala_query_from_file(file_name):
@@ -263,7 +269,8 @@ def exec_hadoop_fs_cmd(args, exit_on_error=True):
exec_cmd(cmd, error_msg="Error executing Hadoop command, exiting",
exit_on_error=exit_on_error)
def exec_query_files_parallel(thread_pool, query_files, execution_type):
def exec_query_files_parallel(thread_pool, query_files, execution_type, step_name):
"""Executes the query files provided using the execution engine specified
in parallel using the given thread pool. Aborts immediately if any execution
encounters an error."""
@@ -274,16 +281,23 @@ def exec_query_files_parallel(thread_pool, query_files, execution_type):
elif execution_type == 'hive':
execution_function = exec_hive_query_from_file_beeline
LOG.info('Begin step "%s".' % step_name)
start_time = time.time()
for result in thread_pool.imap_unordered(execution_function, query_files):
if not result:
thread_pool.terminate()
sys.exit(1)
total_time = time.time() - start_time
LOG.info('End step "%s". Total time: %.2fs\n' % (step_name, total_time))
def impala_exec_query_files_parallel(thread_pool, query_files):
exec_query_files_parallel(thread_pool, query_files, 'impala')
def hive_exec_query_files_parallel(thread_pool, query_files):
exec_query_files_parallel(thread_pool, query_files, 'hive')
def impala_exec_query_files_parallel(thread_pool, query_files, step_name):
exec_query_files_parallel(thread_pool, query_files, 'impala', step_name)
def hive_exec_query_files_parallel(thread_pool, query_files, step_name):
exec_query_files_parallel(thread_pool, query_files, 'hive', step_name)
def main():
logging.basicConfig(format='%(asctime)s %(message)s', datefmt='%H:%M:%S')
@@ -446,38 +460,40 @@ def main():
# so they're done at the end. Finally, the Hbase Tables that have been filled with data
# need to be flushed.
impala_exec_query_files_parallel(thread_pool, impala_create_files)
impala_exec_query_files_parallel(thread_pool, impala_create_files, "Impala Create")
# There should be at most one hbase creation script
assert(len(hbase_create_files) <= 1)
for hbase_create in hbase_create_files:
exec_hbase_query_from_file(hbase_create)
exec_hbase_query_from_file(hbase_create, "HBase Create")
# If this is loading text tables plus multiple other formats, the text tables
# need to be loaded first
assert(len(hive_load_text_files) <= 1)
hive_exec_query_files_parallel(thread_pool, hive_load_text_files)
hive_exec_query_files_parallel(thread_pool, hive_load_text_files, "Hive Load Text")
# IMPALA-9923: Run ORC serially separately from other non-text formats. This hacks
# around flakiness seen when loading this in parallel. This should be removed as
# soon as possible.
# around flakiness seen when loading this in parallel (see IMPALA-12630 comments for
# broken tests). This should be removed as soon as possible.
assert(len(hive_load_orc_files) <= 1)
hive_exec_query_files_parallel(thread_pool, hive_load_orc_files)
hive_exec_query_files_parallel(thread_pool, hive_load_orc_files, "Hive Load ORC")
# Load all non-text formats (goes parallel)
hive_exec_query_files_parallel(thread_pool, hive_load_nontext_files)
hive_exec_query_files_parallel(thread_pool, hive_load_nontext_files,
"Hive Load Non-Text")
assert(len(hbase_postload_files) <= 1)
for hbase_postload in hbase_postload_files:
exec_hbase_query_from_file(hbase_postload)
exec_hbase_query_from_file(hbase_postload, "HBase Post-Load")
# Invalidate so that Impala sees the loads done by Hive before loading Parquet/Kudu
# Note: This only invalidates tables for this workload.
assert(len(invalidate_files) <= 1)
if impala_load_files:
impala_exec_query_files_parallel(thread_pool, invalidate_files)
impala_exec_query_files_parallel(thread_pool, impala_load_files)
impala_exec_query_files_parallel(thread_pool, invalidate_files,
"Impala Invalidate 1")
impala_exec_query_files_parallel(thread_pool, impala_load_files, "Impala Load")
# Final invalidate for this workload
impala_exec_query_files_parallel(thread_pool, invalidate_files)
impala_exec_query_files_parallel(thread_pool, invalidate_files, "Impala Invalidate 2")
loading_time_map[workload] = time.time() - start_time
total_time = 0.0

View File

@@ -1410,7 +1410,8 @@ public class PlannerTest extends PlannerTestBase {
options.setProcessing_cost_min_threads(2);
options.setMax_fragment_instances_per_node(16);
runPlannerTestFile(
"tpcds-processing-cost", "tpcds_parquet", options, tpcdsParquetTestOptions());
"tpcds-processing-cost", "tpcds_partitioned_parquet_snap", options,
tpcdsParquetTestOptions());
}
/**

View File

@@ -44,5 +44,6 @@ if [ "${TARGET_FILESYSTEM}" = "hdfs" ]; then
fi
${COMPUTE_STATS_SCRIPT} --db_names=tpch,tpch_parquet,tpch_orc_def \
--table_names=customer,lineitem,nation,orders,part,partsupp,region,supplier
${COMPUTE_STATS_SCRIPT} --db_names=tpch_nested_parquet,tpcds,tpcds_parquet
${COMPUTE_STATS_SCRIPT} \
--db_names=tpch_nested_parquet,tpcds,tpcds_parquet,tpcds_partitioned_parquet_snap
${COMPUTE_STATS_SCRIPT} --db_names=functional_kudu,tpch_kudu

View File

@@ -234,12 +234,12 @@ function load-data {
TABLE_FORMATS=${3:-}
FORCE_LOAD=${4:-}
MSG="Loading workload '$WORKLOAD'"
LOAD_MSG="Loading workload '$WORKLOAD'"
ARGS=("--workloads $WORKLOAD")
MSG+=" using exploration strategy '$EXPLORATION_STRATEGY'"
LOAD_MSG+=" using exploration strategy '$EXPLORATION_STRATEGY'"
ARGS+=("-e $EXPLORATION_STRATEGY")
if [ $TABLE_FORMATS ]; then
MSG+=" in table formats '$TABLE_FORMATS'"
LOAD_MSG+=" in table formats '$TABLE_FORMATS'"
ARGS+=("--table_formats $TABLE_FORMATS")
fi
if [ $LOAD_DATA_ARGS ]; then
@@ -282,7 +282,7 @@ function load-data {
fi
LOG_FILE=${IMPALA_DATA_LOADING_LOGS_DIR}/${LOG_BASENAME}
echo "$MSG. Logging to ${LOG_FILE}"
echo "$LOAD_MSG. Logging to ${LOG_FILE}"
# Use unbuffered logging by executing with -u
if ! impala-python -u ${IMPALA_HOME}/bin/load-data.py ${ARGS[@]} &> ${LOG_FILE}; then
echo Error loading data. The end of the log file is:
@@ -291,6 +291,11 @@ function load-data {
fi
}
function load-tpcds-data {
load-data "tpcds" "core"
load-data "tpcds_partitioned" "core"
}
function cache-test-tables {
echo CACHING tpch.nation AND functional.alltypestiny
# uncaching the tables first makes this operation idempotent.
@@ -570,7 +575,7 @@ if [ $SKIP_METADATA_LOAD -eq 0 ]; then
run-step-backgroundable "Loading functional-query data" load-functional-query.log \
load-data "functional-query" "exhaustive"
run-step-backgroundable "Loading TPC-H data" load-tpch.log load-data "tpch" "core"
run-step-backgroundable "Loading TPC-DS data" load-tpcds.log load-data "tpcds" "core"
run-step-backgroundable "Loading TPC-DS data" load-tpcds.log load-tpcds-data
run-step-wait-all
# Load tpch nested data.
# TODO: Hacky and introduces more complexity into the system, but it is expedient.

View File

@@ -96,23 +96,19 @@
#
from __future__ import absolute_import, division, print_function
from builtins import object
import collections
import csv
import glob
import json
import math
import os
import random
import re
import shutil
import subprocess
import sys
import tempfile
from itertools import product
from optparse import OptionParser
from tests.common.environ import HIVE_MAJOR_VERSION
from tests.util.test_file_parser import *
from tests.common.test_dimensions import *
from tests.util.test_file_parser import parse_table_constraints, parse_test_file
from tests.common.test_dimensions import (
FILE_FORMAT_TO_STORED_AS_MAP, TableFormatInfo, get_dataset_from_workload,
load_table_info_dimension)
parser = OptionParser()
parser.add_option("-e", "--exploration_strategy", dest="exploration_strategy",
@@ -195,20 +191,6 @@ AVRO_COMPRESSION_MAP = {
'none': '',
}
FILE_FORMAT_MAP = {
'text': 'TEXTFILE',
'seq': 'SEQUENCEFILE',
'rc': 'RCFILE',
'orc': 'ORC',
'parquet': 'PARQUET',
'hudiparquet': 'HUDIPARQUET',
'avro': 'AVRO',
'hbase': "'org.apache.hadoop.hive.hbase.HBaseStorageHandler'",
'kudu': "KUDU",
'iceberg': "ICEBERG",
'json': "JSONFILE",
}
HIVE_TO_AVRO_TYPE_MAP = {
'STRING': 'string',
'INT': 'int',
@@ -240,6 +222,14 @@ WITH SERDEPROPERTIES (
KNOWN_EXPLORATION_STRATEGIES = ['core', 'pairwise', 'exhaustive']
PARTITIONED_INSERT_RE = re.compile(
# Capture multi insert specification.
# Each group represent partition column, min value, max value,
# and num partition per insert.
r'-- partitioned_insert: ([a-z_]+),(\d+),(\d+),(\d+)')
HINT_SHUFFLE = "/* +shuffle, clustered */"
def build_create_statement(table_template, table_name, db_name, db_suffix,
file_format, compression, hdfs_location,
force_reload):
@@ -257,11 +247,12 @@ def build_create_statement(table_template, table_name, db_name, db_suffix,
# Remove location part from the format string
table_template = table_template.replace("LOCATION '{hdfs_location}'", "")
create_stmt += table_template.format(db_name=db_name,
db_suffix=db_suffix,
table_name=table_name,
file_format=FILE_FORMAT_MAP[file_format],
hdfs_location=hdfs_location)
create_stmt += table_template.format(
db_name=db_name,
db_suffix=db_suffix,
table_name=table_name,
file_format=FILE_FORMAT_TO_STORED_AS_MAP[file_format],
hdfs_location=hdfs_location)
return create_stmt
@@ -473,17 +464,59 @@ def build_impala_parquet_codec_statement(codec):
parquet_codec = IMPALA_PARQUET_COMPRESSION_MAP[codec]
return IMPALA_COMPRESSION_CODEC % parquet_codec
def build_partitioned_load(insert, re_match, can_hint, params):
part_col = re_match.group(1)
min_val = int(re_match.group(2))
max_val = int(re_match.group(3))
batch = int(re_match.group(4))
insert = PARTITIONED_INSERT_RE.sub("", insert)
statements = []
params["hint"] = HINT_SHUFFLE if can_hint else ''
first_part = min_val
while first_part < max_val:
if first_part + batch >= max_val:
# This is the last batch
if first_part == min_val:
# There is only 1 batch in this insert. No predicate needed.
params["part_predicate"] = ''
else:
# Insert the remaining partitions + NULL partition
params["part_predicate"] = "WHERE {0} <= {1} OR {2} IS NULL".format(
first_part, part_col, part_col)
elif first_part == min_val:
# This is the first batch.
params["part_predicate"] = "WHERE {0} < {1}".format(
part_col, first_part + batch)
else:
# This is the middle batch.
params["part_predicate"] = "WHERE {0} <= {1} AND {2} < {3}".format(
first_part, part_col, part_col, first_part + batch)
statements.append(insert.format(**params))
first_part += batch
return "\n".join(statements)
def build_insert_into_statement(insert, db_name, db_suffix, table_name, file_format,
hdfs_path, for_impala=False):
insert_hint = ""
if for_impala and (file_format == 'parquet' or is_iceberg_table(file_format)):
insert_hint = "/* +shuffle, clustered */"
insert_statement = insert.format(db_name=db_name,
db_suffix=db_suffix,
table_name=table_name,
hdfs_location=hdfs_path,
impala_home=os.getenv("IMPALA_HOME"),
hint=insert_hint)
can_hint = for_impala and (file_format == 'parquet' or is_iceberg_table(file_format))
params = {
"db_name": db_name,
"db_suffix": db_suffix,
"table_name": table_name,
"hdfs_location": hdfs_path,
"impala_home": os.getenv("IMPALA_HOME"),
"hint": "",
"part_predicate": ""
}
m = PARTITIONED_INSERT_RE.search(insert)
if m:
insert_statement = build_partitioned_load(insert, m, can_hint, params)
else:
if can_hint:
params["hint"] = HINT_SHUFFLE
insert_statement = insert.format(**params)
# Kudu tables are managed and don't support OVERWRITE, so we replace OVERWRITE
# with INTO to make this a regular INSERT.
@@ -539,6 +572,7 @@ def build_insert(insert, db_name, db_suffix, file_format,
for_impala) + "\n"
return output
def build_load_statement(load_template, db_name, db_suffix, table_name):
# hbase does not need the hdfs path.
if table_name.startswith('hbase'):
@@ -547,12 +581,24 @@ def build_load_statement(load_template, db_name, db_suffix, table_name):
db_suffix=db_suffix)
else:
base_load_dir = os.getenv("REMOTE_LOAD", os.getenv("IMPALA_HOME"))
load_template = load_template.format(table_name=table_name,
db_name=db_name,
db_suffix=db_suffix,
impala_home = base_load_dir)
params = {
"db_name": db_name,
"db_suffix": db_suffix,
"table_name": table_name,
"impala_home": base_load_dir,
"hint": "",
"part_predicate": ""
}
m = PARTITIONED_INSERT_RE.search(load_template)
if m:
load_template = build_partitioned_load(load_template, m, False, params)
else:
load_template = load_template.format(**params)
return load_template
def build_hbase_create_stmt(db_name, table_name, column_families, region_splits):
hbase_table_name = "{db_name}_hbase.{table_name}".format(db_name=db_name,
table_name=table_name)

View File

@@ -306,30 +306,6 @@ OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
---- DATASET
tpcds
---- BASE_TABLE_NAME
inventory
---- COLUMNS
inv_date_sk int
inv_item_sk bigint
inv_warehouse_sk int
inv_quantity_on_hand int
primary key (inv_date_sk, inv_item_sk, inv_warehouse_sk) DISABLE NOVALIDATE RELY
foreign key (inv_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (inv_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (inv_warehouse_sk) references {db_name}{db_suffix}.warehouse (w_warehouse_sk) DISABLE NOVALIDATE RELY
---- ROW_FORMAT
delimited fields terminated by '|'
---- TABLE_PROPERTIES
text:serialization.null.format=
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM {db_name}.{table_name};
---- LOAD
LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/inventory/'
OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
====
---- DATASET
tpcds
---- BASE_TABLE_NAME
call_center
---- COLUMNS
cc_call_center_sk int
@@ -536,6 +512,117 @@ OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
---- DATASET
tpcds
---- BASE_TABLE_NAME
web_page
---- COLUMNS
wp_web_page_sk int
wp_web_page_id string
wp_rec_start_date string
wp_rec_end_date string
wp_creation_date_sk int
wp_access_date_sk int
wp_autogen_flag string
wp_customer_sk int
wp_url string
wp_type string
wp_char_count int
wp_link_count int
wp_image_count int
wp_max_ad_count int
primary key (wp_web_page_sk) DISABLE NOVALIDATE RELY
foreign key (wp_creation_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (wp_access_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (wp_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
---- ROW_FORMAT
delimited fields terminated by '|'
---- TABLE_PROPERTIES
text:serialization.null.format=
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM {db_name}.{table_name};
---- LOAD
LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/web_page/'
OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
====
---- DATASET
tpcds
---- BASE_TABLE_NAME
web_site
---- COLUMNS
web_site_sk int
web_site_id string
web_rec_start_date string
web_rec_end_date string
web_name string
web_open_date_sk int
web_close_date_sk int
web_class string
web_manager string
web_mkt_id int
web_mkt_class string
web_mkt_desc string
web_market_manager string
web_company_id int
web_company_name string
web_street_number string
web_street_name string
web_street_type string
web_suite_number string
web_city string
web_county string
web_state string
web_zip string
web_country string
web_gmt_offset decimal(5,2)
web_tax_percentage decimal(5,2)
primary key (web_site_sk) DISABLE NOVALIDATE RELY
foreign key (web_open_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (web_close_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
---- ROW_FORMAT
delimited fields terminated by '|'
---- TABLE_PROPERTIES
text:serialization.null.format=
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM {db_name}.{table_name};
---- LOAD
LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/web_site/'
OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
====
---- DATASET
tpcds
---- BASE_TABLE_NAME
inventory
---- COLUMNS
inv_date_sk int
inv_item_sk bigint
inv_warehouse_sk int
inv_quantity_on_hand int
primary key (inv_date_sk, inv_item_sk, inv_warehouse_sk) DISABLE NOVALIDATE RELY
foreign key (inv_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (inv_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (inv_warehouse_sk) references {db_name}{db_suffix}.warehouse (w_warehouse_sk) DISABLE NOVALIDATE RELY
---- ROW_FORMAT
delimited fields terminated by '|'
---- TABLE_PROPERTIES
text:serialization.null.format=
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM {db_name}.{table_name};
---- LOAD
set hive.exec.max.dynamic.partitions.pernode=10000;
set hive.exec.max.dynamic.partitions=10000;
set hive.exec.dynamic.partition.mode=nonstrict;
set hive.exec.dynamic.partition=true;
set hive.optimize.sort.dynamic.partition=true;
set hive.optimize.sort.dynamic.partition.threshold=1;
set hive.exec.reducers.max=32;
LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/inventory/'
OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
====
---- DATASET
tpcds
---- BASE_TABLE_NAME
catalog_sales
---- COLUMNS
cs_sold_date_sk int
@@ -747,8 +834,8 @@ delimited fields terminated by '|'
---- TABLE_PROPERTIES
text:serialization.null.format=
---- DEPENDENT_LOAD
-- Split the load into multiple steps to reduce total memory usage for larger
-- scale factors. TODO: Dynamically scale this based on the scale factor?
-- partitioned_insert: ss_sold_date_sk,2450800,2452700,450
-- TODO: Dynamically scale this based on the scale factor?
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date_sk)
{hint} SELECT ss_sold_time_sk,
ss_item_sk,
@@ -774,127 +861,12 @@ INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date
ss_net_profit,
ss_sold_date_sk
FROM {db_name}.{table_name}
WHERE ss_sold_date_sk IS NULL;
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date_sk)
{hint} SELECT ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
FROM {db_name}.{table_name}
WHERE ss_sold_date_sk < 2451272;
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date_sk)
{hint} SELECT ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
FROM {db_name}.{table_name}
WHERE 2451272 <= ss_sold_date_sk and ss_sold_date_sk < 2451728;
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date_sk)
{hint} SELECT ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
FROM {db_name}.{table_name}
WHERE 2451728 <= ss_sold_date_sk and ss_sold_date_sk < 2452184;
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date_sk)
{hint} SELECT ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
FROM {db_name}.{table_name}
WHERE 2452184 <= ss_sold_date_sk;
{part_predicate};
---- LOAD
USE {db_name};
set hive.exec.max.dynamic.partitions.pernode=10000;
set hive.exec.max.dynamic.partitions=10000;
set hive.exec.dynamic.partition.mode=nonstrict;
set hive.exec.dynamic.partition=true;
set hive.optimize.sort.dynamic.partition=true;
set hive.optimize.sort.dynamic.partition.threshold=1;
insert overwrite table {table_name} partition(ss_sold_date_sk)
select ss_sold_time_sk,
-- partitioned_insert: ss_sold_date_sk,2450800,2452700,450
-- TODO: Dynamically scale this based on the scale factor?
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date_sk)
SELECT ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
@@ -917,121 +889,9 @@ select ss_sold_time_sk,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
from store_sales_unpartitioned
WHERE ss_sold_date_sk IS NULL
distribute by ss_sold_date_sk;
insert overwrite table {table_name} partition(ss_sold_date_sk)
select ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
from store_sales_unpartitioned
WHERE ss_sold_date_sk < 2451272
distribute by ss_sold_date_sk;
insert overwrite table {table_name} partition(ss_sold_date_sk)
select ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
from store_sales_unpartitioned
WHERE 2451272 <= ss_sold_date_sk and ss_sold_date_sk < 2451728
distribute by ss_sold_date_sk;
insert overwrite table {table_name} partition(ss_sold_date_sk)
select ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
from store_sales_unpartitioned
WHERE 2451728 <= ss_sold_date_sk and ss_sold_date_sk < 2452184
distribute by ss_sold_date_sk;
insert overwrite table {table_name} partition(ss_sold_date_sk)
select ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
from store_sales_unpartitioned
WHERE 2452184 <= ss_sold_date_sk
distribute by ss_sold_date_sk;
FROM {db_name}.store_sales_unpartitioned
{part_predicate}
DISTRIBUTE BY ss_sold_date_sk;
====
---- DATASET
tpcds
@@ -1083,85 +943,6 @@ OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
---- DATASET
tpcds
---- BASE_TABLE_NAME
web_page
---- COLUMNS
wp_web_page_sk int
wp_web_page_id string
wp_rec_start_date string
wp_rec_end_date string
wp_creation_date_sk int
wp_access_date_sk int
wp_autogen_flag string
wp_customer_sk int
wp_url string
wp_type string
wp_char_count int
wp_link_count int
wp_image_count int
wp_max_ad_count int
primary key (wp_web_page_sk) DISABLE NOVALIDATE RELY
foreign key (wp_creation_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (wp_access_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (wp_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
---- ROW_FORMAT
delimited fields terminated by '|'
---- TABLE_PROPERTIES
text:serialization.null.format=
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM {db_name}.{table_name};
---- LOAD
LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/web_page/'
OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
====
---- DATASET
tpcds
---- BASE_TABLE_NAME
web_site
---- COLUMNS
web_site_sk int
web_site_id string
web_rec_start_date string
web_rec_end_date string
web_name string
web_open_date_sk int
web_close_date_sk int
web_class string
web_manager string
web_mkt_id int
web_mkt_class string
web_mkt_desc string
web_market_manager string
web_company_id int
web_company_name string
web_street_number string
web_street_name string
web_street_type string
web_suite_number string
web_city string
web_county string
web_state string
web_zip string
web_country string
web_gmt_offset decimal(5,2)
web_tax_percentage decimal(5,2)
primary key (web_site_sk) DISABLE NOVALIDATE RELY
foreign key (web_open_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (web_close_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
---- ROW_FORMAT
delimited fields terminated by '|'
---- TABLE_PROPERTIES
text:serialization.null.format=
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM {db_name}.{table_name};
---- LOAD
LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/web_site/'
OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
====
---- DATASET
tpcds
---- BASE_TABLE_NAME
web_sales
---- COLUMNS
ws_sold_date_sk int

View File

@@ -0,0 +1,5 @@
This dataset is a fully-partitioned version of tpcds dataset.
This dataset can not be loaded independently and requires the base
'tpcds' db from the tpcds dataset to be preloaded first. An example of
how to load this dataset can be seen at function load-tpcds-data in
bin/create-load-data.sh.

View File

@@ -0,0 +1,4 @@
# Table level constraints:
# Allows for defining constraints on which file formats to generate for an individual
# table. The table name should match the base table name defined in the schema template
# file.
1 # Table level constraints:
2 # Allows for defining constraints on which file formats to generate for an individual
3 # table. The table name should match the base table name defined in the schema template
4 # file.

View File

@@ -0,0 +1,978 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you 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.
#
# For details on this file format please see ../README
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
time_dim
---- COLUMNS
t_time_sk int
t_time_id string
t_time int
t_hour int
t_minute int
t_second int
t_am_pm string
t_shift string
t_sub_shift string
t_meal_time string
primary key (t_time_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
date_dim
---- COLUMNS
d_date_sk int
d_date_id string
d_date string
d_month_seq int
d_week_seq int
d_quarter_seq int
d_year int
d_dow int
d_moy int
d_dom int
d_qoy int
d_fy_year int
d_fy_quarter_seq int
d_fy_week_seq int
d_day_name string
d_quarter_name string
d_holiday string
d_weekend string
d_following_holiday string
d_first_dom int
d_last_dom int
d_same_day_ly int
d_same_day_lq int
d_current_day string
d_current_week string
d_current_month string
d_current_quarter string
d_current_year string
primary key (d_date_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
reason
---- COLUMNS
r_reason_sk int
r_reason_id string
r_reason_desc string
primary key (r_reason_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
customer_address
---- COLUMNS
ca_address_sk int
ca_address_id string
ca_street_number string
ca_street_name string
ca_street_type string
ca_suite_number string
ca_city string
ca_county string
ca_state string
ca_zip string
ca_country string
ca_gmt_offset decimal(5,2)
ca_location_type string
primary key (ca_address_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
income_band
---- COLUMNS
ib_income_band_sk int
ib_lower_bound int
ib_upper_bound int
primary key (ib_income_band_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
household_demographics
---- COLUMNS
hd_demo_sk int
hd_income_band_sk int
hd_buy_potential string
hd_dep_count int
hd_vehicle_count int
primary key (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (hd_income_band_sk) references {db_name}{db_suffix}.income_band (ib_income_band_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
customer_demographics
---- COLUMNS
cd_demo_sk int
cd_gender string
cd_marital_status string
cd_education_status string
cd_purchase_estimate int
cd_credit_rating string
cd_dep_count int
cd_dep_employed_count int
cd_dep_college_count int
primary key (cd_demo_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
ship_mode
---- COLUMNS
sm_ship_mode_sk int
sm_ship_mode_id string
sm_type string
sm_code string
sm_carrier string
sm_contract string
primary key (sm_ship_mode_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
item
---- COLUMNS
i_item_sk bigint
i_item_id string
i_rec_start_date string
i_rec_end_date string
i_item_desc string
i_current_price decimal(7,2)
i_wholesale_cost decimal(7,2)
i_brand_id int
i_brand string
i_class_id int
i_class string
i_category_id int
i_category string
i_manufact_id int
i_manufact string
i_size string
i_formulation string
i_color string
i_units string
i_container string
i_manager_id int
i_product_name string
primary key (i_item_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
warehouse
---- COLUMNS
w_warehouse_sk int
w_warehouse_id string
w_warehouse_name string
w_warehouse_sq_ft int
w_street_number string
w_street_name string
w_street_type string
w_suite_number string
w_city string
w_county string
w_state string
w_zip string
w_country string
w_gmt_offset decimal(5,2)
primary key (w_warehouse_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
call_center
---- COLUMNS
cc_call_center_sk int
cc_call_center_id string
cc_rec_start_date string
cc_rec_end_date string
cc_closed_date_sk int
cc_open_date_sk int
cc_name string
cc_class string
cc_employees int
cc_sq_ft int
cc_hours string
cc_manager string
cc_mkt_id int
cc_mkt_class string
cc_mkt_desc string
cc_market_manager string
cc_division int
cc_division_name string
cc_company int
cc_company_name string
cc_street_number string
cc_street_name string
cc_street_type string
cc_suite_number string
cc_city string
cc_county string
cc_state string
cc_zip string
cc_country string
cc_gmt_offset decimal(5,2)
cc_tax_percentage decimal(5,2)
primary key (cc_call_center_sk) DISABLE NOVALIDATE RELY
foreign key (cc_closed_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (cc_open_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
customer
---- COLUMNS
c_customer_sk int
c_customer_id string
c_current_cdemo_sk int
c_current_hdemo_sk int
c_current_addr_sk int
c_first_shipto_date_sk int
c_first_sales_date_sk int
c_salutation string
c_first_name string
c_last_name string
c_preferred_cust_flag string
c_birth_day int
c_birth_month int
c_birth_year int
c_birth_country string
c_login string
c_email_address string
c_last_review_date string
primary key (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (c_current_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (c_current_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (c_current_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (c_first_sales_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (c_first_shipto_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
promotion
---- COLUMNS
p_promo_sk int
p_promo_id string
p_start_date_sk int
p_end_date_sk int
p_item_sk bigint
p_cost decimal(15,2)
p_response_target int
p_promo_name string
p_channel_dmail string
p_channel_email string
p_channel_catalog string
p_channel_tv string
p_channel_radio string
p_channel_press string
p_channel_event string
p_channel_demo string
p_channel_details string
p_purpose string
p_discount_active string
primary key (p_promo_sk) DISABLE NOVALIDATE RELY
foreign key (p_start_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (p_end_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (p_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
store
---- COLUMNS
s_store_sk int
s_store_id string
s_rec_start_date string
s_rec_end_date string
s_closed_date_sk int
s_store_name string
s_number_employees int
s_floor_space int
s_hours string
s_manager string
s_market_id int
s_geography_class string
s_market_desc string
s_market_manager string
s_division_id int
s_division_name string
s_company_id int
s_company_name string
s_street_number string
s_street_name string
s_street_type string
s_suite_number string
s_city string
s_county string
s_state string
s_zip string
s_country string
s_gmt_offset decimal(5,2)
s_tax_precentage decimal(5,2)
primary key (s_store_sk) DISABLE NOVALIDATE RELY
foreign key (s_closed_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
catalog_page
---- COLUMNS
cp_catalog_page_sk int
cp_catalog_page_id string
cp_start_date_sk int
cp_end_date_sk int
cp_department string
cp_catalog_number int
cp_catalog_page_number int
cp_description string
cp_type string
primary key (cp_catalog_page_sk) DISABLE NOVALIDATE RELY
foreign key (cp_start_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (cp_end_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
web_page
---- COLUMNS
wp_web_page_sk int
wp_web_page_id string
wp_rec_start_date string
wp_rec_end_date string
wp_creation_date_sk int
wp_access_date_sk int
wp_autogen_flag string
wp_customer_sk int
wp_url string
wp_type string
wp_char_count int
wp_link_count int
wp_image_count int
wp_max_ad_count int
primary key (wp_web_page_sk) DISABLE NOVALIDATE RELY
foreign key (wp_creation_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (wp_access_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (wp_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
web_site
---- COLUMNS
web_site_sk int
web_site_id string
web_rec_start_date string
web_rec_end_date string
web_name string
web_open_date_sk int
web_close_date_sk int
web_class string
web_manager string
web_mkt_id int
web_mkt_class string
web_mkt_desc string
web_market_manager string
web_company_id int
web_company_name string
web_street_number string
web_street_name string
web_street_type string
web_suite_number string
web_city string
web_county string
web_state string
web_zip string
web_country string
web_gmt_offset decimal(5,2)
web_tax_percentage decimal(5,2)
primary key (web_site_sk) DISABLE NOVALIDATE RELY
foreign key (web_open_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (web_close_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
SELECT * FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
inventory
---- COLUMNS
inv_item_sk bigint
inv_warehouse_sk int
inv_quantity_on_hand int
primary key (inv_item_sk, inv_warehouse_sk) DISABLE NOVALIDATE RELY
foreign key (inv_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (inv_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (inv_warehouse_sk) references {db_name}{db_suffix}.warehouse (w_warehouse_sk) DISABLE NOVALIDATE RELY
---- PARTITION_COLUMNS
inv_date_sk int
---- DEPENDENT_LOAD
-- The following query options are set to optimize small scale fact tables with 3 impalads.
-- Small MAX_SCAN_RANGE_LENGTH will split 1 file into few scan ranges that can be
-- distributed across 3 impalads in minicluster. Without this, only 1 fragment at 1
-- impalad does the reading and writing.
SET MAX_SCAN_RANGE_LENGTH=1mb;
-- MT_DOP=4 is to increase number of scanner and writer to 12 at max for single insert
-- overwrite query.
SET MT_DOP=4;
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} partition(inv_date_sk)
SELECT
inv_item_sk,
inv_warehouse_sk,
inv_quantity_on_hand,
inv_date_sk
FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
catalog_sales
---- COLUMNS
cs_sold_time_sk int
cs_ship_date_sk int
cs_bill_customer_sk int
cs_bill_cdemo_sk int
cs_bill_hdemo_sk int
cs_bill_addr_sk int
cs_ship_customer_sk int
cs_ship_cdemo_sk int
cs_ship_hdemo_sk int
cs_ship_addr_sk int
cs_call_center_sk int
cs_catalog_page_sk int
cs_ship_mode_sk int
cs_warehouse_sk int
cs_item_sk bigint
cs_promo_sk int
cs_order_number bigint
cs_quantity int
cs_wholesale_cost decimal(7,2)
cs_list_price decimal(7,2)
cs_sales_price decimal(7,2)
cs_ext_discount_amt decimal(7,2)
cs_ext_sales_price decimal(7,2)
cs_ext_wholesale_cost decimal(7,2)
cs_ext_list_price decimal(7,2)
cs_ext_tax decimal(7,2)
cs_coupon_amt decimal(7,2)
cs_ext_ship_cost decimal(7,2)
cs_net_paid decimal(7,2)
cs_net_paid_inc_tax decimal(7,2)
cs_net_paid_inc_ship decimal(7,2)
cs_net_paid_inc_ship_tax decimal(7,2)
cs_net_profit decimal(7,2)
primary key (cs_item_sk, cs_order_number) DISABLE NOVALIDATE RELY
foreign key (cs_sold_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (cs_sold_time_sk) references {db_name}{db_suffix}.time_dim (t_time_sk) DISABLE NOVALIDATE RELY
foreign key (cs_ship_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (cs_bill_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (cs_bill_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cs_bill_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cs_bill_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (cs_ship_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (cs_ship_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cs_ship_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cs_ship_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (cs_call_center_sk) references {db_name}{db_suffix}.call_center (cc_call_center_sk) DISABLE NOVALIDATE RELY
foreign key (cs_catalog_page_sk) references {db_name}{db_suffix}.catalog_page (cp_catalog_page_sk) DISABLE NOVALIDATE RELY
foreign key (cs_ship_mode_sk) references {db_name}{db_suffix}.ship_mode (sm_ship_mode_sk) DISABLE NOVALIDATE RELY
foreign key (cs_warehouse_sk) references {db_name}{db_suffix}.warehouse (w_warehouse_sk) DISABLE NOVALIDATE RELY
foreign key (cs_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (cs_promo_sk) references {db_name}{db_suffix}.promotion (p_promo_sk) DISABLE NOVALIDATE RELY
---- PARTITION_COLUMNS
cs_sold_date_sk int
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} partition(cs_sold_date_sk)
SELECT
cs_sold_time_sk,
cs_ship_date_sk,
cs_bill_customer_sk,
cs_bill_cdemo_sk,
cs_bill_hdemo_sk,
cs_bill_addr_sk,
cs_ship_customer_sk,
cs_ship_cdemo_sk,
cs_ship_hdemo_sk,
cs_ship_addr_sk,
cs_call_center_sk,
cs_catalog_page_sk,
cs_ship_mode_sk,
cs_warehouse_sk,
cs_item_sk,
cs_promo_sk,
cs_order_number,
cs_quantity,
cs_wholesale_cost,
cs_list_price,
cs_sales_price,
cs_ext_discount_amt,
cs_ext_sales_price,
cs_ext_wholesale_cost,
cs_ext_list_price,
cs_ext_tax,
cs_coupon_amt,
cs_ext_ship_cost,
cs_net_paid,
cs_net_paid_inc_tax,
cs_net_paid_inc_ship,
cs_net_paid_inc_ship_tax,
cs_net_profit,
cs_sold_date_sk
FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
catalog_returns
---- COLUMNS
cr_returned_time_sk int
cr_item_sk bigint
cr_refunded_customer_sk int
cr_refunded_cdemo_sk int
cr_refunded_hdemo_sk int
cr_refunded_addr_sk int
cr_returning_customer_sk int
cr_returning_cdemo_sk int
cr_returning_hdemo_sk int
cr_returning_addr_sk int
cr_call_center_sk int
cr_catalog_page_sk int
cr_ship_mode_sk int
cr_warehouse_sk int
cr_reason_sk int
cr_order_number bigint
cr_return_quantity int
cr_return_amount decimal(7,2)
cr_return_tax decimal(7,2)
cr_return_amt_inc_tax decimal(7,2)
cr_fee decimal(7,2)
cr_return_ship_cost decimal(7,2)
cr_refunded_cash decimal(7,2)
cr_reversed_charge decimal(7,2)
cr_store_credit decimal(7,2)
cr_net_loss decimal(7,2)
primary key (cr_item_sk, cr_order_number) DISABLE NOVALIDATE RELY
foreign key (cr_returned_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (cr_returned_time_sk) references {db_name}{db_suffix}.time_dim (t_time_sk) DISABLE NOVALIDATE RELY
foreign key (cr_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (cr_refunded_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (cr_refunded_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cr_refunded_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cr_refunded_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (cr_returning_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (cr_returning_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cr_returning_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (cr_returning_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (cr_call_center_sk) references {db_name}{db_suffix}.call_center (cc_call_center_sk) DISABLE NOVALIDATE RELY
foreign key (cr_catalog_page_sk) references {db_name}{db_suffix}.catalog_page (cp_catalog_page_sk) DISABLE NOVALIDATE RELY
foreign key (cr_ship_mode_sk) references {db_name}{db_suffix}.ship_mode (sm_ship_mode_sk) DISABLE NOVALIDATE RELY
foreign key (cr_warehouse_sk) references {db_name}{db_suffix}.warehouse (w_warehouse_sk) DISABLE NOVALIDATE RELY
foreign key (cr_reason_sk) references {db_name}{db_suffix}.reason (r_reason_sk) DISABLE NOVALIDATE RELY
foreign key (cr_item_sk, cr_order_number) references {db_name}{db_suffix}.catalog_sales (cs_item_sk, cs_order_number) DISABLE NOVALIDATE RELY
---- PARTITION_COLUMNS
cr_returned_date_sk int
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} partition(cr_returned_date_sk)
SELECT
cr_returned_time_sk,
cr_item_sk,
cr_refunded_customer_sk,
cr_refunded_cdemo_sk,
cr_refunded_hdemo_sk,
cr_refunded_addr_sk,
cr_returning_customer_sk,
cr_returning_cdemo_sk,
cr_returning_hdemo_sk,
cr_returning_addr_sk,
cr_call_center_sk,
cr_catalog_page_sk,
cr_ship_mode_sk,
cr_warehouse_sk,
cr_reason_sk,
cr_order_number,
cr_return_quantity,
cr_return_amount,
cr_return_tax,
cr_return_amt_inc_tax,
cr_fee,
cr_return_ship_cost,
cr_refunded_cash,
cr_reversed_charge,
cr_store_credit,
cr_net_loss,
cr_returned_date_sk
FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
store_sales
---- COLUMNS
ss_sold_time_sk int
ss_item_sk bigint
ss_customer_sk int
ss_cdemo_sk int
ss_hdemo_sk int
ss_addr_sk int
ss_store_sk int
ss_promo_sk int
ss_ticket_number bigint
ss_quantity int
ss_wholesale_cost decimal(7,2)
ss_list_price decimal(7,2)
ss_sales_price decimal(7,2)
ss_ext_discount_amt decimal(7,2)
ss_ext_sales_price decimal(7,2)
ss_ext_wholesale_cost decimal(7,2)
ss_ext_list_price decimal(7,2)
ss_ext_tax decimal(7,2)
ss_coupon_amt decimal(7,2)
ss_net_paid decimal(7,2)
ss_net_paid_inc_tax decimal(7,2)
ss_net_profit decimal(7,2)
primary key (ss_item_sk, ss_ticket_number) DISABLE NOVALIDATE RELY
foreign key (ss_sold_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (ss_sold_time_sk) references {db_name}{db_suffix}.time_dim (t_time_sk) DISABLE NOVALIDATE RELY
foreign key (ss_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (ss_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (ss_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (ss_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (ss_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (ss_store_sk) references {db_name}{db_suffix}.store (s_store_sk) DISABLE NOVALIDATE RELY
foreign key (ss_promo_sk) references {db_name}{db_suffix}.promotion (p_promo_sk) DISABLE NOVALIDATE RELY
---- PARTITION_COLUMNS
ss_sold_date_sk int
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (ss_sold_date_sk)
SELECT ss_sold_time_sk,
ss_item_sk,
ss_customer_sk,
ss_cdemo_sk,
ss_hdemo_sk,
ss_addr_sk,
ss_store_sk,
ss_promo_sk,
ss_ticket_number,
ss_quantity,
ss_wholesale_cost,
ss_list_price,
ss_sales_price,
ss_ext_discount_amt,
ss_ext_sales_price,
ss_ext_wholesale_cost,
ss_ext_list_price,
ss_ext_tax,
ss_coupon_amt,
ss_net_paid,
ss_net_paid_inc_tax,
ss_net_profit,
ss_sold_date_sk
FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
store_returns
---- COLUMNS
sr_return_time_sk int
sr_item_sk bigint
sr_customer_sk int
sr_cdemo_sk int
sr_hdemo_sk int
sr_addr_sk int
sr_store_sk int
sr_reason_sk int
sr_ticket_number bigint
sr_return_quantity int
sr_return_amt decimal(7,2)
sr_return_tax decimal(7,2)
sr_return_amt_inc_tax decimal(7,2)
sr_fee decimal(7,2)
sr_return_ship_cost decimal(7,2)
sr_refunded_cash decimal(7,2)
sr_reversed_charge decimal(7,2)
sr_store_credit decimal(7,2)
sr_net_loss decimal(7,2)
primary key (sr_item_sk, sr_ticket_number) DISABLE NOVALIDATE RELY
foreign key (sr_returned_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (sr_return_time_sk) references {db_name}{db_suffix}.time_dim (t_time_sk) DISABLE NOVALIDATE RELY
foreign key (sr_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (sr_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (sr_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (sr_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (sr_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (sr_store_sk) references {db_name}{db_suffix}.store (s_store_sk) DISABLE NOVALIDATE RELY
foreign key (sr_reason_sk) references {db_name}{db_suffix}.reason (r_reason_sk) DISABLE NOVALIDATE RELY
foreign key (sr_item_sk, sr_ticket_number) references {db_name}{db_suffix}.store_sales (ss_item_sk, ss_ticket_number) DISABLE NOVALIDATE RELY
---- PARTITION_COLUMNS
sr_returned_date_sk int
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} partition(sr_returned_date_sk)
SELECT
sr_return_time_sk,
sr_item_sk,
sr_customer_sk,
sr_cdemo_sk,
sr_hdemo_sk,
sr_addr_sk,
sr_store_sk,
sr_reason_sk,
sr_ticket_number,
sr_return_quantity,
sr_return_amt,
sr_return_tax,
sr_return_amt_inc_tax,
sr_fee,
sr_return_ship_cost,
sr_refunded_cash,
sr_reversed_charge,
sr_store_credit,
sr_net_loss,
sr_returned_date_sk
FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
web_sales
---- COLUMNS
ws_sold_time_sk int
ws_ship_date_sk int
ws_item_sk bigint
ws_bill_customer_sk int
ws_bill_cdemo_sk int
ws_bill_hdemo_sk int
ws_bill_addr_sk int
ws_ship_customer_sk int
ws_ship_cdemo_sk int
ws_ship_hdemo_sk int
ws_ship_addr_sk int
ws_web_page_sk int
ws_web_site_sk int
ws_ship_mode_sk int
ws_warehouse_sk int
ws_promo_sk int
ws_order_number bigint
ws_quantity int
ws_wholesale_cost decimal(7,2)
ws_list_price decimal(7,2)
ws_sales_price decimal(7,2)
ws_ext_discount_amt decimal(7,2)
ws_ext_sales_price decimal(7,2)
ws_ext_wholesale_cost decimal(7,2)
ws_ext_list_price decimal(7,2)
ws_ext_tax decimal(7,2)
ws_coupon_amt decimal(7,2)
ws_ext_ship_cost decimal(7,2)
ws_net_paid decimal(7,2)
ws_net_paid_inc_tax decimal(7,2)
ws_net_paid_inc_ship decimal(7,2)
ws_net_paid_inc_ship_tax decimal(7,2)
ws_net_profit decimal(7,2)
primary key (ws_item_sk, ws_order_number) DISABLE NOVALIDATE RELY
foreign key (ws_sold_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (ws_sold_time_sk) references {db_name}{db_suffix}.time_dim (t_time_sk) DISABLE NOVALIDATE RELY
foreign key (ws_ship_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (ws_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (ws_bill_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (ws_bill_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (ws_bill_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (ws_bill_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (ws_ship_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (ws_ship_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (ws_ship_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (ws_ship_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (ws_web_page_sk) references {db_name}{db_suffix}.web_page (wp_web_page_sk) DISABLE NOVALIDATE RELY
foreign key (ws_web_site_sk) references {db_name}{db_suffix}.web_site (web_site_sk) DISABLE NOVALIDATE RELY
foreign key (ws_ship_mode_sk) references {db_name}{db_suffix}.ship_mode (sm_ship_mode_sk) DISABLE NOVALIDATE RELY
foreign key (ws_warehouse_sk) references {db_name}{db_suffix}.warehouse (w_warehouse_sk) DISABLE NOVALIDATE RELY
foreign key (ws_promo_sk) references {db_name}{db_suffix}.promotion (p_promo_sk) DISABLE NOVALIDATE RELY
---- PARTITION_COLUMNS
ws_sold_date_sk int
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} partition(ws_sold_date_sk)
SELECT
ws_sold_time_sk,
ws_ship_date_sk,
ws_item_sk,
ws_bill_customer_sk,
ws_bill_cdemo_sk,
ws_bill_hdemo_sk,
ws_bill_addr_sk,
ws_ship_customer_sk,
ws_ship_cdemo_sk,
ws_ship_hdemo_sk,
ws_ship_addr_sk,
ws_web_page_sk,
ws_web_site_sk,
ws_ship_mode_sk,
ws_warehouse_sk,
ws_promo_sk,
ws_order_number,
ws_quantity,
ws_wholesale_cost,
ws_list_price,
ws_sales_price,
ws_ext_discount_amt,
ws_ext_sales_price,
ws_ext_wholesale_cost,
ws_ext_list_price,
ws_ext_tax,
ws_coupon_amt,
ws_ext_ship_cost,
ws_net_paid,
ws_net_paid_inc_tax,
ws_net_paid_inc_ship,
ws_net_paid_inc_ship_tax,
ws_net_profit,
ws_sold_date_sk
FROM tpcds.{table_name};
====
---- DATASET
tpcds_partitioned
---- BASE_TABLE_NAME
web_returns
---- COLUMNS
wr_returned_time_sk int
wr_item_sk bigint
wr_refunded_customer_sk int
wr_refunded_cdemo_sk int
wr_refunded_hdemo_sk int
wr_refunded_addr_sk int
wr_returning_customer_sk int
wr_returning_cdemo_sk int
wr_returning_hdemo_sk int
wr_returning_addr_sk int
wr_web_page_sk int
wr_reason_sk int
wr_order_number bigint
wr_return_quantity int
wr_return_amt decimal(7,2)
wr_return_tax decimal(7,2)
wr_return_amt_inc_tax decimal(7,2)
wr_fee decimal(7,2)
wr_return_ship_cost decimal(7,2)
wr_refunded_cash decimal(7,2)
wr_reversed_charge decimal(7,2)
wr_account_credit decimal(7,2)
wr_net_loss decimal(7,2)
primary key (wr_item_sk, wr_order_number) DISABLE NOVALIDATE RELY
foreign key (wr_returned_date_sk) references {db_name}{db_suffix}.date_dim (d_date_sk) DISABLE NOVALIDATE RELY
foreign key (wr_returned_time_sk) references {db_name}{db_suffix}.time_dim (t_time_sk) DISABLE NOVALIDATE RELY
foreign key (wr_item_sk) references {db_name}{db_suffix}.item (i_item_sk) DISABLE NOVALIDATE RELY
foreign key (wr_refunded_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (wr_refunded_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (wr_refunded_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (wr_refunded_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (wr_returning_customer_sk) references {db_name}{db_suffix}.customer (c_customer_sk) DISABLE NOVALIDATE RELY
foreign key (wr_returning_cdemo_sk) references {db_name}{db_suffix}.customer_demographics (cd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (wr_returning_hdemo_sk) references {db_name}{db_suffix}.household_demographics (hd_demo_sk) DISABLE NOVALIDATE RELY
foreign key (wr_returning_addr_sk) references {db_name}{db_suffix}.customer_address (ca_address_sk) DISABLE NOVALIDATE RELY
foreign key (wr_web_page_sk) references {db_name}{db_suffix}.web_page (wp_web_page_sk) DISABLE NOVALIDATE RELY
foreign key (wr_reason_sk) references {db_name}{db_suffix}.reason (r_reason_sk) DISABLE NOVALIDATE RELY
foreign key (wr_item_sk, wr_order_number) references {db_name}{db_suffix}.web_sales (ws_item_sk, ws_order_number) DISABLE NOVALIDATE RELY
---- PARTITION_COLUMNS
wr_returned_date_sk int
---- DEPENDENT_LOAD
INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} partition(wr_returned_date_sk)
SELECT
wr_returned_time_sk,
wr_item_sk,
wr_refunded_customer_sk,
wr_refunded_cdemo_sk,
wr_refunded_hdemo_sk,
wr_refunded_addr_sk,
wr_returning_customer_sk,
wr_returning_cdemo_sk,
wr_returning_hdemo_sk,
wr_returning_addr_sk,
wr_web_page_sk,
wr_reason_sk,
wr_order_number,
wr_return_quantity,
wr_return_amt,
wr_return_tax,
wr_return_amt_inc_tax,
wr_fee,
wr_return_ship_cost,
wr_refunded_cash,
wr_reversed_charge,
wr_account_credit,
wr_net_loss,
wr_returned_date_sk
FROM tpcds.{table_name};
====

View File

@@ -0,0 +1 @@
file_format: parquet, dataset: tpcds_partitioned, compression_codec: snap, compression_type: block
1 file_format: parquet dataset: tpcds_partitioned compression_codec: snap compression_type: block

View File

@@ -0,0 +1,4 @@
file_format: parquet
dataset: tpcds_partitioned
compression_codec: snap
compression_type: block
1 file_format: parquet
2 dataset: tpcds_partitioned
3 compression_codec: snap
4 compression_type: block

View File

@@ -0,0 +1 @@
file_format: parquet, dataset: tpcds_partitioned, compression_codec: snap, compression_type: block
1 file_format: parquet dataset: tpcds_partitioned compression_codec: snap compression_type: block

View File

@@ -0,0 +1 @@
file_format: parquet, dataset: tpcds_partitioned, compression_codec: snap, compression_type: block
1 file_format: parquet dataset: tpcds_partitioned compression_codec: snap compression_type: block