This is an automated email from the ASF dual-hosted git repository.
dkuzmenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new add340d2408 HIVE-27327: Iceberg: Incorrect row count in basic stats
leads to suboptimal plans (Simhadri Govindappa, reviewed by Attila Turoczy,
Butao Zhang, Denys Kuzmenko, Zsolt Miskolczi)
add340d2408 is described below
commit add340d24081931af27f57e522a837382295ddcf
Author: Simhadri Govindappa <[email protected]>
AuthorDate: Mon May 15 15:52:33 2023 +0530
HIVE-27327: Iceberg: Incorrect row count in basic stats leads to
suboptimal plans (Simhadri Govindappa, reviewed by Attila Turoczy, Butao Zhang,
Denys Kuzmenko, Zsolt Miskolczi)
Closes #4301
---
.../iceberg/mr/hive/HiveIcebergStorageHandler.java | 53 +--
.../src/test/queries/positive/row_count.q | 57 +++
.../positive/vectorized_iceberg_merge_mixed.q | 71 +++-
.../src/test/results/positive/col_stats.q.out | 6 +-
.../results/positive/iceberg_copy_on_write.q.out | 6 +-
.../src/test/results/positive/row_count.q.out | 302 +++++++++++++++
.../positive/vectorized_iceberg_merge_mixed.q.out | 403 +++++++++++++++++----
7 files changed, 789 insertions(+), 109 deletions(-)
diff --git
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
index 9ee6874dcbd..a08eafc06d3 100644
---
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
+++
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
@@ -353,32 +353,39 @@ public class HiveIcebergStorageHandler implements
HiveStoragePredicateHandler, H
Table table = getTable(hmsTable);
String statsSource = HiveConf.getVar(conf,
HiveConf.ConfVars.HIVE_ICEBERG_STATS_SOURCE).toLowerCase();
Map<String, String> stats = Maps.newHashMap();
- switch (statsSource) {
- case ICEBERG:
- if (table.currentSnapshot() != null) {
- Map<String, String> summary = table.currentSnapshot().summary();
- if (summary != null) {
- if (summary.containsKey(SnapshotSummary.TOTAL_DATA_FILES_PROP)) {
- stats.put(StatsSetupConst.NUM_FILES,
summary.get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
- }
- if (summary.containsKey(SnapshotSummary.TOTAL_RECORDS_PROP)) {
- stats.put(StatsSetupConst.ROW_COUNT,
summary.get(SnapshotSummary.TOTAL_RECORDS_PROP));
- }
- if (summary.containsKey(SnapshotSummary.TOTAL_FILE_SIZE_PROP)) {
- stats.put(StatsSetupConst.TOTAL_SIZE,
summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP));
+ if (statsSource.equals(ICEBERG)) {
+ if (table.currentSnapshot() != null) {
+ Map<String, String> summary = table.currentSnapshot().summary();
+ if (summary != null) {
+
+ if (summary.containsKey(SnapshotSummary.TOTAL_DATA_FILES_PROP)) {
+ stats.put(StatsSetupConst.NUM_FILES,
summary.get(SnapshotSummary.TOTAL_DATA_FILES_PROP));
+ }
+
+ if (summary.containsKey(SnapshotSummary.TOTAL_RECORDS_PROP)) {
+ long totalRecords =
Long.parseLong(summary.get(SnapshotSummary.TOTAL_RECORDS_PROP));
+ if (summary.containsKey(SnapshotSummary.TOTAL_EQ_DELETES_PROP) &&
+ summary.containsKey(SnapshotSummary.TOTAL_POS_DELETES_PROP)) {
+
+ long totalEqDeletes =
Long.parseLong(summary.get(SnapshotSummary.TOTAL_EQ_DELETES_PROP));
+ long totalPosDeletes =
Long.parseLong(summary.get(SnapshotSummary.TOTAL_POS_DELETES_PROP));
+
+ long actualRecords = totalRecords - (totalEqDeletes > 0 ? 0 :
totalPosDeletes);
+ totalRecords = actualRecords > 0 ? actualRecords : totalRecords;
+ // actualRecords maybe -ve in edge cases
}
+ stats.put(StatsSetupConst.ROW_COUNT, String.valueOf(totalRecords));
+ }
+
+ if (summary.containsKey(SnapshotSummary.TOTAL_FILE_SIZE_PROP)) {
+ stats.put(StatsSetupConst.TOTAL_SIZE,
summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP));
}
- } else {
- stats.put(StatsSetupConst.NUM_FILES, "0");
- stats.put(StatsSetupConst.ROW_COUNT, "0");
- stats.put(StatsSetupConst.TOTAL_SIZE, "0");
}
- break;
- case PUFFIN:
- // place holder for puffin
- break;
- default:
- // fall back to metastore
+ } else {
+ stats.put(StatsSetupConst.NUM_FILES, "0");
+ stats.put(StatsSetupConst.ROW_COUNT, "0");
+ stats.put(StatsSetupConst.TOTAL_SIZE, "0");
+ }
}
return stats;
}
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/row_count.q
b/iceberg/iceberg-handler/src/test/queries/positive/row_count.q
new file mode 100644
index 00000000000..02e57f4d302
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/queries/positive/row_count.q
@@ -0,0 +1,57 @@
+-- Mask random uuid
+--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
+-- Mask random snapshot id
+--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#SnapshotId#/
+-- Mask current-snapshot-timestamp-ms
+--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#/
+-- Mask totalSize
+--! qt:replace:/(\s+totalSize\s+)\S+(\s*)/$1#Masked#/
+-- Mask added file size
+--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask total file size
+--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
+-- Mask width
+--! qt:replace:/(width=15)\d+/$1###/
+
+drop table if exists llap_orders;
+
+CREATE EXTERNAL TABLE llap_orders (orderid INT, quantity INT, itemid INT,
tradets TIMESTAMP) PARTITIONED BY (p1 STRING, p2 STRING) STORED BY ICEBERG
STORED AS ORC tblproperties('format-version'='2');
+
+
+INSERT INTO llap_orders VALUES
+(0, 48, 5, timestamp('2000-06-04 19:55:46.129'), 'EU', 'DE'),
+(1, 12, 6, timestamp('2007-06-24 19:23:22.829'), 'US', 'TX'),
+(2, 76, 4, timestamp('2018-02-19 23:43:51.995'), 'EU', 'DE'),
+(3, 91, 5, timestamp('2000-07-15 09:09:11.587'), 'US', 'NJ'),
+(4, 18, 6, timestamp('2007-12-02 22:30:39.302'), 'EU', 'ES'),
+(5, 71, 5, timestamp('2010-02-08 20:31:23.430'), 'EU', 'DE'),
+(6, 78, 3, timestamp('2016-02-22 20:37:37.025'), 'EU', 'FR'),
+(7, 88, 0, timestamp('2020-03-26 18:47:40.611'), 'EU', 'FR'),
+(8, 87, 4, timestamp('2003-02-20 00:48:09.139'), 'EU', 'ES'),
+(9, 60, 6, timestamp('2012-08-28 01:35:54.283'), 'EU', 'IT'),
+(10, 24, 5, timestamp('2015-03-28 18:57:50.069'), 'US', 'NY'),
+(11, 42, 2, timestamp('2012-06-27 01:13:32.350'), 'EU', 'UK'),
+(12, 37, 4, timestamp('2020-08-09 01:18:50.153'), 'US', 'NY'),
+(13, 52, 1, timestamp('2019-09-04 01:46:19.558'), 'EU', 'UK'),
+(14, 96, 3, timestamp('2019-03-05 22:00:03.020'), 'US', 'NJ'),
+(15, 18, 3, timestamp('2001-09-11 00:14:12.687'), 'EU', 'FR'),
+(16, 46, 0, timestamp('2013-08-31 02:16:17.878'), 'EU', 'UK'),
+(17, 26, 5, timestamp('2001-02-01 20:05:32.317'), 'EU', 'FR'),
+(18, 68, 5, timestamp('2009-12-29 08:44:08.048'), 'EU', 'ES'),
+(19, 54, 6, timestamp('2015-08-15 01:59:22.177'), 'EU', 'HU'),
+(20, 10, 0, timestamp('2018-05-06 12:56:12.789'), 'US', 'CA');
+
+--check row count
+select count(*) from llap_orders;
+describe formatted llap_orders;
+
+--delete rows
+delete from llap_orders where itemid = 6;
+delete from llap_orders where itemid = 5;
+
+--check for updated row count
+select count(*) from llap_orders;
+describe formatted llap_orders;
+
+explain select count(*) from llap_orders;
+explain insert into llap_orders select * from llap_orders limit 100000;
diff --git
a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q
b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q
index 4b61c99825f..3d14c5134ee 100644
---
a/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q
+++
b/iceberg/iceberg-handler/src/test/queries/positive/vectorized_iceberg_merge_mixed.q
@@ -2,6 +2,10 @@
-- Mask neededVirtualColumns due to non-strict order
--! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/
+-- Mask width
+--! qt:replace:/(width=17)\d+/$1####/
+-- Mask total data size
+--! qt:replace:/(Data size: 35)\d+/$1####/
set hive.vectorized.execution.enabled=true;
set hive.llap.io.enabled=false;
@@ -42,6 +46,8 @@ stored by ICEBERG stored as PARQUET
insert into store_sales (ss_customer_sk, ss_item_sk, ss_sold_date_sk) values
(1,1501,"2451181"), (2,1502,"2451181"), (3,1503,"2451181"), (4,1504,"2451181"),
(5,1505,"2451181");
delete from store_sales where ss_customer_sk > 2;
+select count(*) from store_sales;
+
create table ssv (
ss_sold_date_sk int,
ss_sold_time_sk int,
@@ -73,6 +79,8 @@ stored by ICEBERG stored as ORC
insert into ssv (ss_customer_sk2, ss_item_sk2, ss_ext_discount_amt) values
(1,1501,-0.1), (2,1502,-0.1), (3,1503,-0.1), (4,1504,-0.1), (5,1505,-0.1);
+select count(*) from ssv;
+
explain vectorization detail
MERGE INTO store_sales t
USING ssv s
@@ -135,7 +143,68 @@ WHEN NOT matched THEN
);
select * from store_sales;
-
+
+explain
+MERGE INTO store_sales t
+ USING ssv s
+ON (t.ss_item_sk = s.ss_item_sk2
+ AND t.ss_customer_sk=s.ss_customer_sk2
+ AND t.ss_sold_date_sk = "2451181"
+ AND ((Floor((s.ss_item_sk2) / 1000) * 1000) BETWEEN 1000 AND 2000)
+ AND s.ss_ext_discount_amt < 0.0) WHEN matched
+ AND t.ss_ext_discount_amt IS NULL
+THEN UPDATE
+ SET ss_ext_discount_amt = 0.0
+WHEN NOT matched THEN
+ INSERT (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)
+ VALUES (
+ s.ss_sold_time_sk,
+ s.ss_item_sk2,
+ s.ss_customer_sk2,
+ s.ss_cdemo_sk,
+ s.ss_hdemo_sk,
+ s.ss_addr_sk,
+ s.ss_store_sk,
+ s.ss_promo_sk,
+ s.ss_ticket_number,
+ s.ss_quantity,
+ s.ss_wholesale_cost,
+ s.ss_list_price,
+ s.ss_sales_price,
+ s.ss_ext_discount_amt,
+ s.ss_ext_sales_price,
+ s.ss_ext_wholesale_cost,
+ s.ss_ext_list_price,
+ s.ss_ext_tax,
+ s.ss_coupon_amt,
+ s.ss_net_paid,
+ s.ss_net_paid_inc_tax,
+ s.ss_net_profit,
+ "2451181"
+ );
+
MERGE INTO store_sales t
USING ssv s
ON (t.ss_item_sk = s.ss_item_sk2
diff --git a/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out
b/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out
index 851cb106ac3..c8d5eab3580 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/col_stats.q.out
@@ -138,13 +138,13 @@ Stage-0
Stage-1
Reducer 2 vectorized
File Output Operator [FS_8]
- Select Operator [SEL_7] (rows=24 width=95)
+ Select Operator [SEL_7] (rows=18 width=95)
Output:["_col0","_col1","_col2"]
<-Map 1 [SIMPLE_EDGE] vectorized
SHUFFLE [RS_6]
- Select Operator [SEL_5] (rows=24 width=95)
+ Select Operator [SEL_5] (rows=18 width=95)
Output:["_col0","_col1","_col2"]
- TableScan [TS_0] (rows=24 width=95)
+ TableScan [TS_0] (rows=18 width=95)
default@tbl_ice_puffin,tbl_ice_puffin,Tbl:COMPLETE,Col:COMPLETE,Output:["a","b","c"]
PREHOOK: query: select * from tbl_ice_puffin order by a, b, c
diff --git
a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out
b/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out
index 631812392e6..bccd5c416c3 100644
---
a/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out
+++
b/iceberg/iceberg-handler/src/test/results/positive/iceberg_copy_on_write.q.out
@@ -78,11 +78,11 @@ Stage-3
Map 1 vectorized
File Output Operator [FS_7]
table:{"name:":"default.ice01"}
- Select Operator [SEL_6] (rows=7 width=78)
+ Select Operator [SEL_6] (rows=5 width=91)
Output:["_col0","_col1"]
- Filter Operator [FIL_5] (rows=7 width=78)
+ Filter Operator [FIL_5] (rows=5 width=91)
predicate:(((id <= 4) and (id <> 2)) or ((id > 4) or (id =
2)) is null)
- TableScan [TS_0] (rows=9 width=81)
+ TableScan [TS_0] (rows=7 width=78)
default@ice01,ice01,Tbl:COMPLETE,Col:COMPLETE,Output:["id","name"]
PREHOOK: query: delete from ice01 where id>4 OR id=2
diff --git a/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out
b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out
new file mode 100644
index 00000000000..4035276e0c3
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/results/positive/row_count.q.out
@@ -0,0 +1,302 @@
+PREHOOK: query: drop table if exists llap_orders
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists llap_orders
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE EXTERNAL TABLE llap_orders (orderid INT, quantity INT,
itemid INT, tradets TIMESTAMP) PARTITIONED BY (p1 STRING, p2 STRING) STORED BY
ICEBERG STORED AS ORC tblproperties('format-version'='2')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@llap_orders
+POSTHOOK: query: CREATE EXTERNAL TABLE llap_orders (orderid INT, quantity INT,
itemid INT, tradets TIMESTAMP) PARTITIONED BY (p1 STRING, p2 STRING) STORED BY
ICEBERG STORED AS ORC tblproperties('format-version'='2')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@llap_orders
+PREHOOK: query: INSERT INTO llap_orders VALUES
+(0, 48, 5, timestamp('2000-06-04 19:55:46.129'), 'EU', 'DE'),
+(1, 12, 6, timestamp('2007-06-24 19:23:22.829'), 'US', 'TX'),
+(2, 76, 4, timestamp('2018-02-19 23:43:51.995'), 'EU', 'DE'),
+(3, 91, 5, timestamp('2000-07-15 09:09:11.587'), 'US', 'NJ'),
+(4, 18, 6, timestamp('2007-12-02 22:30:39.302'), 'EU', 'ES'),
+(5, 71, 5, timestamp('2010-02-08 20:31:23.430'), 'EU', 'DE'),
+(6, 78, 3, timestamp('2016-02-22 20:37:37.025'), 'EU', 'FR'),
+(7, 88, 0, timestamp('2020-03-26 18:47:40.611'), 'EU', 'FR'),
+(8, 87, 4, timestamp('2003-02-20 00:48:09.139'), 'EU', 'ES'),
+(9, 60, 6, timestamp('2012-08-28 01:35:54.283'), 'EU', 'IT'),
+(10, 24, 5, timestamp('2015-03-28 18:57:50.069'), 'US', 'NY'),
+(11, 42, 2, timestamp('2012-06-27 01:13:32.350'), 'EU', 'UK'),
+(12, 37, 4, timestamp('2020-08-09 01:18:50.153'), 'US', 'NY'),
+(13, 52, 1, timestamp('2019-09-04 01:46:19.558'), 'EU', 'UK'),
+(14, 96, 3, timestamp('2019-03-05 22:00:03.020'), 'US', 'NJ'),
+(15, 18, 3, timestamp('2001-09-11 00:14:12.687'), 'EU', 'FR'),
+(16, 46, 0, timestamp('2013-08-31 02:16:17.878'), 'EU', 'UK'),
+(17, 26, 5, timestamp('2001-02-01 20:05:32.317'), 'EU', 'FR'),
+(18, 68, 5, timestamp('2009-12-29 08:44:08.048'), 'EU', 'ES'),
+(19, 54, 6, timestamp('2015-08-15 01:59:22.177'), 'EU', 'HU'),
+(20, 10, 0, timestamp('2018-05-06 12:56:12.789'), 'US', 'CA')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@llap_orders
+POSTHOOK: query: INSERT INTO llap_orders VALUES
+(0, 48, 5, timestamp('2000-06-04 19:55:46.129'), 'EU', 'DE'),
+(1, 12, 6, timestamp('2007-06-24 19:23:22.829'), 'US', 'TX'),
+(2, 76, 4, timestamp('2018-02-19 23:43:51.995'), 'EU', 'DE'),
+(3, 91, 5, timestamp('2000-07-15 09:09:11.587'), 'US', 'NJ'),
+(4, 18, 6, timestamp('2007-12-02 22:30:39.302'), 'EU', 'ES'),
+(5, 71, 5, timestamp('2010-02-08 20:31:23.430'), 'EU', 'DE'),
+(6, 78, 3, timestamp('2016-02-22 20:37:37.025'), 'EU', 'FR'),
+(7, 88, 0, timestamp('2020-03-26 18:47:40.611'), 'EU', 'FR'),
+(8, 87, 4, timestamp('2003-02-20 00:48:09.139'), 'EU', 'ES'),
+(9, 60, 6, timestamp('2012-08-28 01:35:54.283'), 'EU', 'IT'),
+(10, 24, 5, timestamp('2015-03-28 18:57:50.069'), 'US', 'NY'),
+(11, 42, 2, timestamp('2012-06-27 01:13:32.350'), 'EU', 'UK'),
+(12, 37, 4, timestamp('2020-08-09 01:18:50.153'), 'US', 'NY'),
+(13, 52, 1, timestamp('2019-09-04 01:46:19.558'), 'EU', 'UK'),
+(14, 96, 3, timestamp('2019-03-05 22:00:03.020'), 'US', 'NJ'),
+(15, 18, 3, timestamp('2001-09-11 00:14:12.687'), 'EU', 'FR'),
+(16, 46, 0, timestamp('2013-08-31 02:16:17.878'), 'EU', 'UK'),
+(17, 26, 5, timestamp('2001-02-01 20:05:32.317'), 'EU', 'FR'),
+(18, 68, 5, timestamp('2009-12-29 08:44:08.048'), 'EU', 'ES'),
+(19, 54, 6, timestamp('2015-08-15 01:59:22.177'), 'EU', 'HU'),
+(20, 10, 0, timestamp('2018-05-06 12:56:12.789'), 'US', 'CA')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@llap_orders
+PREHOOK: query: select count(*) from llap_orders
+PREHOOK: type: QUERY
+PREHOOK: Input: default@llap_orders
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from llap_orders
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@llap_orders
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+21
+PREHOOK: query: describe formatted llap_orders
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@llap_orders
+POSTHOOK: query: describe formatted llap_orders
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@llap_orders
+# col_name data_type comment
+orderid int
+quantity int
+itemid int
+tradets timestamp
+p1 string
+p2 string
+
+# Partition Transform Information
+# col_name transform_type
+p1 IDENTITY
+p2 IDENTITY
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE
{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"itemid\":\"true\",\"orderid\":\"true\",\"p1\":\"true\",\"p2\":\"true\",\"quantity\":\"true\",\"tradets\":\"true\"}}
+ EXTERNAL TRUE
+ bucketing_version 2
+ current-schema
{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"orderid\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"quantity\",\"required\":false,\"type\":\"int\"},{\"id\":3,\"name\":\"itemid\",\"required\":false,\"type\":\"int\"},{\"id\":4,\"name\":\"tradets\",\"required\":false,\"type\":\"timestamp\"},{\"id\":5,\"name\":\"p1\",\"required\":false,\"type\":\"string\"},{\"id\":6,\"name\":\"p2\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #SnapshotId#
+ current-snapshot-summary
{\"added-data-files\":\"10\",\"added-records\":\"21\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"10\",\"total-records\":\"21\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"10\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec
{\"spec-id\":0,\"fields\":[{\"name\":\"p1\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"p2\",\"transform\":\"identity\",\"source-id\":6,\"field-id\":1001}]}
+ engine.hive.enabled true
+ format-version 2
+ iceberg.orc.files.only true
+ metadata_location hdfs://### HDFS PATH ###
+ numFiles 10
+ numRows 21
+ previous_metadata_location hdfs://### HDFS PATH ###
+ rawDataSize 0
+ serialization.format 1
+ snapshot-count 1
+ storage_handler
org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
+ table_type ICEBERG
+ totalSize #Masked#
+#### A masked pattern was here ####
+ uuid #Masked#
+ write.delete.mode merge-on-read
+ write.format.default orc
+ write.merge.mode merge-on-read
+ write.update.mode merge-on-read
+
+# Storage Information
+SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+Compressed: No
+Sort Columns: []
+PREHOOK: query: delete from llap_orders where itemid = 6
+PREHOOK: type: QUERY
+PREHOOK: Input: default@llap_orders
+PREHOOK: Output: default@llap_orders
+POSTHOOK: query: delete from llap_orders where itemid = 6
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@llap_orders
+POSTHOOK: Output: default@llap_orders
+PREHOOK: query: delete from llap_orders where itemid = 5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@llap_orders
+PREHOOK: Output: default@llap_orders
+POSTHOOK: query: delete from llap_orders where itemid = 5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@llap_orders
+POSTHOOK: Output: default@llap_orders
+PREHOOK: query: select count(*) from llap_orders
+PREHOOK: type: QUERY
+PREHOOK: Input: default@llap_orders
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from llap_orders
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@llap_orders
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+11
+PREHOOK: query: describe formatted llap_orders
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@llap_orders
+POSTHOOK: query: describe formatted llap_orders
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@llap_orders
+# col_name data_type comment
+orderid int
+quantity int
+itemid int
+tradets timestamp
+p1 string
+p2 string
+
+# Partition Transform Information
+# col_name transform_type
+p1 IDENTITY
+p2 IDENTITY
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
+ EXTERNAL TRUE
+ bucketing_version 2
+ current-schema
{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"orderid\",\"required\":false,\"type\":\"int\"},{\"id\":2,\"name\":\"quantity\",\"required\":false,\"type\":\"int\"},{\"id\":3,\"name\":\"itemid\",\"required\":false,\"type\":\"int\"},{\"id\":4,\"name\":\"tradets\",\"required\":false,\"type\":\"timestamp\"},{\"id\":5,\"name\":\"p1\",\"required\":false,\"type\":\"string\"},{\"id\":6,\"name\":\"p2\",\"required\":false,\"type\":\"string\"}]}
+ current-snapshot-id #SnapshotId#
+ current-snapshot-summary
{\"added-position-delete-files\":\"5\",\"added-delete-files\":\"5\",\"added-files-size\":\"#Masked#\",\"added-position-deletes\":\"6\",\"changed-partition-count\":\"5\",\"total-records\":\"21\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"10\",\"total-delete-files\":\"9\",\"total-position-deletes\":\"10\",\"total-equality-deletes\":\"0\"}
+ current-snapshot-timestamp-ms #Masked#
+ default-partition-spec
{\"spec-id\":0,\"fields\":[{\"name\":\"p1\",\"transform\":\"identity\",\"source-id\":5,\"field-id\":1000},{\"name\":\"p2\",\"transform\":\"identity\",\"source-id\":6,\"field-id\":1001}]}
+ engine.hive.enabled true
+ format-version 2
+ iceberg.orc.files.only true
+ metadata_location hdfs://### HDFS PATH ###
+ numFiles 10
+ numRows 11
+ previous_metadata_location hdfs://### HDFS PATH ###
+ rawDataSize 0
+ serialization.format 1
+ snapshot-count 3
+ storage_handler
org.apache.iceberg.mr.hive.HiveIcebergStorageHandler
+ table_type ICEBERG
+ totalSize #Masked#
+#### A masked pattern was here ####
+ uuid #Masked#
+ write.delete.mode merge-on-read
+ write.format.default orc
+ write.merge.mode merge-on-read
+ write.update.mode merge-on-read
+
+# Storage Information
+SerDe Library: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+InputFormat: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+OutputFormat: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+Compressed: No
+Sort Columns: []
+PREHOOK: query: explain select count(*) from llap_orders
+PREHOOK: type: QUERY
+PREHOOK: Input: default@llap_orders
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: explain select count(*) from llap_orders
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@llap_orders
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+
+Stage-0
+ Fetch Operator
+ limit:-1
+ Stage-1
+ Reducer 2 vectorized
+ File Output Operator [FS_11]
+ Group By Operator [GBY_10] (rows=1 width=8)
+ Output:["_col0"],aggregations:["count(VALUE._col0)"]
+ <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
+ PARTITION_ONLY_SHUFFLE [RS_9]
+ Group By Operator [GBY_8] (rows=1 width=8)
+ Output:["_col0"],aggregations:["count()"]
+ Select Operator [SEL_7] (rows=11 width=15###)
+ TableScan [TS_0] (rows=11 width=15###)
+ default@llap_orders,llap_orders,Tbl:COMPLETE,Col:COMPLETE
+
+PREHOOK: query: explain insert into llap_orders select * from llap_orders
limit 100000
+PREHOOK: type: QUERY
+PREHOOK: Input: default@llap_orders
+PREHOOK: Output: default@llap_orders
+POSTHOOK: query: explain insert into llap_orders select * from llap_orders
limit 100000
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@llap_orders
+POSTHOOK: Output: default@llap_orders
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+
+Stage-3
+ Stats Work{}
+ Stage-0
+ Move Operator
+ table:{"name:":"default.llap_orders"}
+ Stage-2
+ Dependency Collection{}
+ Stage-1
+ Reducer 3 vectorized
+ File Output Operator [FS_26]
+ table:{"name:":"default.llap_orders"}
+ Select Operator [SEL_25]
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col4","_col5"]
+ <-Reducer 2 [SIMPLE_EDGE] vectorized
+ PARTITION_ONLY_SHUFFLE [RS_21]
+ PartitionCols:_col4, _col5
+ Select Operator [SEL_20] (rows=11 width=224)
+ Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+ Limit [LIM_19] (rows=11 width=224)
+ Number of rows:100000
+ <-Map 1 [CUSTOM_SIMPLE_EDGE] vectorized
+ PARTITION_ONLY_SHUFFLE [RS_18]
+ Select Operator [SEL_17] (rows=11 width=224)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
+ Limit [LIM_16] (rows=11 width=224)
+ Number of rows:100000
+ TableScan [TS_0] (rows=11 width=224)
+
default@llap_orders,llap_orders,Tbl:COMPLETE,Col:COMPLETE,Output:["orderid","quantity","itemid","tradets","p1","p2"]
+ Reducer 4 vectorized
+ File Output Operator [FS_29]
+ Select Operator [SEL_28] (rows=1 width=1657)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35"]
+ Group By Operator [GBY_27] (rows=1 width=1048)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24"],aggregations:["min(VALUE._col0)","max(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","compute_bit_vector_hll(VALUE._col4)","min(VALUE._col5)","max(VALUE._col6)","count(VALUE._col7)","compute_bit_vector_hll(VALUE._col8)","min(VALUE._col9)",
[...]
+ <-Reducer 2 [CUSTOM_SIMPLE_EDGE] vectorized
+ PARTITION_ONLY_SHUFFLE [RS_24]
+ Group By Operator [GBY_23] (rows=1 width=1184)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24"],aggregations:["min(orderid)","max(orderid)","count(1)","count(orderid)","compute_bit_vector_hll(orderid)","min(quantity)","max(quantity)","count(quantity)","compute_bit_vector_hll(quantity)","min(itemid)","max(itemid)","count(itemid)","compute_
[...]
+ Select Operator [SEL_22] (rows=11 width=224)
+
Output:["orderid","quantity","itemid","tradets","p1","p2"]
+ Please refer to the previous Select Operator
[SEL_20]
+
diff --git
a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out
b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out
index 2a21ab6fbe6..ea4af1de462 100644
---
a/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out
+++
b/iceberg/iceberg-handler/src/test/results/positive/vectorized_iceberg_merge_mixed.q.out
@@ -84,6 +84,15 @@ POSTHOOK: query: delete from store_sales where
ss_customer_sk > 2
POSTHOOK: type: QUERY
POSTHOOK: Input: default@store_sales
POSTHOOK: Output: default@store_sales
+PREHOOK: query: select count(*) from store_sales
+PREHOOK: type: QUERY
+PREHOOK: Input: default@store_sales
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from store_sales
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+2
PREHOOK: query: create table ssv (
ss_sold_date_sk int,
ss_sold_time_sk int,
@@ -154,6 +163,15 @@ POSTHOOK: query: insert into ssv (ss_customer_sk2,
ss_item_sk2, ss_ext_discount_
POSTHOOK: type: QUERY
POSTHOOK: Input: _dummy_database@_dummy_table
POSTHOOK: Output: default@ssv
+PREHOOK: query: select count(*) from ssv
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ssv
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select count(*) from ssv
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ssv
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+5
PREHOOK: query: explain vectorization detail
MERGE INTO store_sales t
USING ssv s
@@ -303,69 +321,14 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Map 2 <- Map 1 (BROADCAST_EDGE)
- Reducer 3 <- Map 2 (SIMPLE_EDGE)
- Reducer 4 <- Map 2 (SIMPLE_EDGE)
- Reducer 5 <- Map 2 (SIMPLE_EDGE)
- Reducer 6 <- Map 2 (SIMPLE_EDGE)
+ Map 1 <- Map 6 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE)
+ Reducer 4 <- Map 1 (SIMPLE_EDGE)
+ Reducer 5 <- Map 1 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
- Map Operator Tree:
- TableScan
- alias: store_sales
- filterExpr: ((ss_sold_date_sk = 2451181) and ss_item_sk is
not null and ss_customer_sk is not null) (type: boolean)
- Statistics: Num rows: 5 Data size: 1436 Basic stats:
COMPLETE Column stats: COMPLETE
- TableScan Vectorization:
- native: true
- vectorizationSchemaColumns: [0:ss_sold_date_sk:int,
1:ss_sold_time_sk:int, 2:ss_item_sk:int, 3:ss_customer_sk:int,
4:ss_cdemo_sk:int, 5:ss_hdemo_sk:int, 6:ss_addr_sk:int, 7:ss_store_sk:int,
8:ss_promo_sk:int, 9:ss_ticket_number:int, 10:ss_quantity:int,
11:ss_wholesale_cost:decimal(7,2), 12:ss_list_price:decimal(7,2),
13:ss_sales_price:decimal(7,2), 14:ss_ext_discount_amt:decimal(7,2),
15:ss_ext_sales_price:decimal(7,2), 16:ss_ext_wholesale_cost:decimal(7,2),
17:ss_e [...]
- Filter Operator
- Filter Vectorization:
- className: VectorFilterOperator
- native: true
- predicateExpression: FilterExprAndExpr(children:
FilterLongColEqualLongScalar(col 0:int, val 2451181), SelectColumnIsNotNull(col
2:int), SelectColumnIsNotNull(col 3:int))
- predicate: ((ss_sold_date_sk = 2451181) and ss_item_sk is
not null and ss_customer_sk is not null) (type: boolean)
- Statistics: Num rows: 5 Data size: 1436 Basic stats:
COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: PARTITION__SPEC__ID (type: int),
PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type:
bigint), 2451181 (type: int), ss_sold_time_sk (type: int), ss_item_sk (type:
int), ss_customer_sk (type: int), ss_cdemo_sk (type: int), ss_hdemo_sk (type:
int), ss_addr_sk (type: int), ss_store_sk (type: int), ss_promo_sk (type: int),
ss_ticket_number (type: int), ss_quantity (type: int), ss_wholesale_cost (type:
decimal(7,2)), ss_list_price [...]
- outputColumnNames: _col0, _col1, _col2, _col3, _col4,
_col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14,
_col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24,
_col25, _col26
- Select Vectorization:
- className: VectorSelectOperator
- native: true
- projectedOutputColumnNums: [23, 24, 25, 26, 27, 1,
2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22]
- selectExpressions: ConstantVectorExpression(val
2451181) -> 27:int
- Statistics: Num rows: 5 Data size: 2456 Basic stats:
COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col6 (type: int), _col7 (type: int)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col6 (type: int), _col7
(type: int)
- Reduce Sink Vectorization:
- className: VectorReduceSinkMultiKeyOperator
- keyColumns: 2:int, 3:int
- native: true
- nativeConditionsMet:
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true,
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
- valueColumns: 23:int, 24:bigint, 25:string,
26:bigint, 27:int, 1:int, 4:int, 5:int, 6:int, 7:int, 8:int, 9:int, 10:int,
11:decimal(7,2), 12:decimal(7,2), 13:decimal(7,2), 14:decimal(7,2),
15:decimal(7,2), 16:decimal(7,2), 17:decimal(7,2), 18:decimal(7,2),
19:decimal(7,2), 20:decimal(7,2), 21:decimal(7,2), 22:decimal(7,2)
- Statistics: Num rows: 5 Data size: 2456 Basic stats:
COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: int), _col1 (type:
bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5
(type: int), _col8 (type: int), _col9 (type: int), _col10 (type: int), _col11
(type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col15
(type: decimal(7,2)), _col16 (type: decimal(7,2)), _col17 (type: decimal(7,2)),
_col18 (type: decimal(7,2)), _col19 (type: decimal(7,2)), _col20 (type:
decimal(7,2)), _col21 (type: [...]
- Execution mode: vectorized
- Map Vectorization:
- enabled: true
- enabledConditionsMet:
hive.vectorized.use.vectorized.input.format IS true
- inputFormatFeatureSupport: []
- featureSupportInUse: []
- inputFileFormats:
org.apache.iceberg.mr.hive.HiveIcebergInputFormat
- allNative: true
- usesVectorUDFAdaptor: false
- vectorized: true
- rowBatchContext:
- dataColumnCount: 23
- includeColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12,
13, 14, 15, 16, 17, 18, 19, 20, 21, 22]
- dataColumns: ss_sold_date_sk:int, ss_sold_time_sk:int,
ss_item_sk:int, 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:int,
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 [...]
- neededVirtualColumns: #Masked#
- partitionColumnCount: 0
- scratchColumnTypeNames: [bigint]
- Map 2
Map Operator Tree:
TableScan
alias: s
@@ -384,13 +347,13 @@ STAGE PLANS:
Statistics: Num rows: 5 Data size: 1904 Basic stats:
COMPLETE Column stats: COMPLETE
Map Join Operator
condition map:
- Right Outer Join 0 to 1
+ Left Outer Join 0 to 1
filter predicates:
- 0
- 1 {_col22} {_col23}
+ 0 {_col22} {_col23}
+ 1
keys:
- 0 _col6 (type: int), _col7 (type: int)
- 1 _col1 (type: int), _col2 (type: int)
+ 0 _col1 (type: int), _col2 (type: int)
+ 1 _col6 (type: int), _col7 (type: int)
Map Join Vectorization:
bigTableFilterExpressions: SelectColumnIsTrue(col
31:boolean), SelectColumnIsTrue(col 32:boolean)
bigTableKeyColumns: 2:int, 3:int
@@ -400,21 +363,21 @@ STAGE PLANS:
native: true
nativeConditionsMet:
hive.mapjoin.optimized.hashtable IS true,
hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine
tez IN [tez] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small
table vectorizes IS true, Outer Join has keys IS true, Optimized Table and
Supports Key Types IS true
outerSmallTableKeyMapping: 2 -> 39, 3 -> 40
- projectedOutput: 33:int, 34:bigint, 35:string,
36:bigint, 37:int, 38:int, 39:int, 40:int, 41:int, 42:int, 43:int, 44:int,
45:int, 46:int, 47:int, 48:decimal(7,2), 49:decimal(7,2), 50:decimal(7,2),
51:decimal(7,2), 52:decimal(7,2), 53:decimal(7,2), 54:decimal(7,2),
55:decimal(7,2), 56:decimal(7,2), 57:decimal(7,2), 58:decimal(7,2),
59:decimal(7,2), 1:int, 2:int, 3:int, 4:int, 5:int, 6:int, 7:int, 8:int, 9:int,
10:int, 11:decimal(7,2), 12:decimal(7,2), 13:decimal( [...]
+ projectedOutput: 1:int, 2:int, 3:int, 4:int, 5:int,
6:int, 7:int, 8:int, 9:int, 10:int, 11:decimal(7,2), 12:decimal(7,2),
13:decimal(7,2), 14:decimal(7,2), 15:decimal(7,2), 16:decimal(7,2),
17:decimal(7,2), 18:decimal(7,2), 19:decimal(7,2), 20:decimal(7,2),
21:decimal(7,2), 22:decimal(7,2), 33:int, 34:bigint, 35:string, 36:bigint,
37:int, 38:int, 39:int, 40:int, 41:int, 42:int, 43:int, 44:int, 45:int, 46:int,
47:int, 48:decimal(7,2), 49:decimal(7,2), 50:decimal( [...]
smallTableValueMapping: 33:int, 34:bigint,
35:string, 36:bigint, 37:int, 38:int, 41:int, 42:int, 43:int, 44:int, 45:int,
46:int, 47:int, 48:decimal(7,2), 49:decimal(7,2), 50:decimal(7,2),
51:decimal(7,2), 52:decimal(7,2), 53:decimal(7,2), 54:decimal(7,2),
55:decimal(7,2), 56:decimal(7,2), 57:decimal(7,2), 58:decimal(7,2),
59:decimal(7,2)
hashTableImplementationType: OPTIMIZED
- outputColumnNames: _col0, _col1, _col2, _col3, _col4,
_col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14,
_col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24,
_col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34,
_col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44,
_col45, _col46, _col47, _col48
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4,
_col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14,
_col15, _col16, _col17, _col18, _col19, _col20, _col21, _col24, _col25, _col26,
_col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36,
_col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46,
_col47, _col48, _col49, _col50
input vertices:
- 0 Map 1
- Statistics: Num rows: 5 Data size: 4320 Basic stats:
COMPLETE Column stats: COMPLETE
+ 1 Map 6
+ Statistics: Num rows: 7 Data size: 6656 Basic stats:
COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col15 (type: decimal(7,2)), _col46
(type: decimal(7,2)), _col8 (type: int), _col14 (type: int), _col0 (type: int),
_col48 (type: decimal(7,2)), _col2 (type: string), _col34 (type: int), _col30
(type: int), _col37 (type: decimal(7,2)), _col24 (type: decimal(7,2)), _col13
(type: int), _col33 (type: int), _col25 (type: decimal(7,2)), _col38 (type:
decimal(7,2)), _col9 (type: int), _col44 (type: decimal(7,2)), _col41 (type:
decimal(7,2)), _col20 (type: d [...]
+ expressions: _col39 (type: decimal(7,2)), _col19
(type: decimal(7,2)), _col32 (type: int), _col38 (type: int), _col24 (type:
int), _col21 (type: decimal(7,2)), _col26 (type: string), _col7 (type: int),
_col3 (type: int), _col10 (type: decimal(7,2)), _col48 (type: decimal(7,2)),
_col37 (type: int), _col6 (type: int), _col49 (type: decimal(7,2)), _col11
(type: decimal(7,2)), _col33 (type: int), _col17 (type: decimal(7,2)), _col14
(type: decimal(7,2)), _col44 (type: [...]
outputColumnNames: _col0, _col1, _col2, _col3, _col4,
_col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14,
_col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24,
_col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34,
_col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44,
_col45, _col46, _col47, _col48
Select Vectorization:
className: VectorSelectOperator
native: true
projectedOutputColumnNums: [48, 20, 41, 47, 33,
22, 35, 8, 4, 11, 57, 46, 7, 58, 12, 42, 18, 15, 53, 55, 21, 9, 43, 39, 49, 44,
5, 56, 6, 14, 13, 52, 51, 2, 17, 37, 3, 16, 19, 34, 10, 38, 50, 1, 36, 59, 40,
45, 54]
- Statistics: Num rows: 5 Data size: 4320 Basic stats:
COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 7 Data size: 6656 Basic stats:
COMPLETE Column stats: COMPLETE
Filter Operator
Filter Vectorization:
className: VectorFilterOperator
@@ -478,7 +441,7 @@ STAGE PLANS:
native: true
predicateExpression: FilterExprAndExpr(children:
SelectColumnIsNull(col 39:int), SelectColumnIsNull(col 40:int),
SelectColumnIsNull(col 37:int))
predicate: (_col23 is null and _col46 is null and
_col35 is null) (type: boolean)
- Statistics: Num rows: 1 Data size: 2976 Basic stats:
COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 4696 Basic stats:
COMPLETE Column stats: COMPLETE
Select Operator
expressions: 2451181 (type: int), _col43 (type:
int), _col33 (type: int), _col36 (type: int), _col8 (type: int), _col26 (type:
int), _col28 (type: int), _col12 (type: int), _col7 (type: int), _col21 (type:
int), _col40 (type: int), _col9 (type: decimal(7,2)), _col14 (type:
decimal(7,2)), _col30 (type: decimal(7,2)), _col29 (type: decimal(7,2)), _col17
(type: decimal(7,2)), _col37 (type: decimal(7,2)), _col34 (type: decimal(7,2)),
_col16 (type: decimal(7,2)), _ [...]
outputColumnNames: _col0, _col1, _col2, _col3,
_col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13,
_col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22
@@ -487,7 +450,7 @@ STAGE PLANS:
native: true
projectedOutputColumnNums: [30, 1, 2, 3, 4, 5,
6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22]
selectExpressions:
ConstantVectorExpression(val 2451181) -> 30:int
- Statistics: Num rows: 1 Data size: 1388 Basic
stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 2900 Basic
stats: COMPLETE Column stats: COMPLETE
Reduce Output Operator
key expressions: _col3 (type: int),
iceberg_bucket(_col2, 3) (type: int)
null sort order: aa
@@ -500,7 +463,7 @@ STAGE PLANS:
native: true
nativeConditionsMet:
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true,
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
valueColumns: 30:int, 1:int, 2:int, 3:int,
4:int, 5:int, 6:int, 7:int, 8:int, 9:int, 10:int, 11:decimal(7,2),
12:decimal(7,2), 13:decimal(7,2), 14:decimal(7,2), 15:decimal(7,2),
16:decimal(7,2), 17:decimal(7,2), 18:decimal(7,2), 19:decimal(7,2),
20:decimal(7,2), 21:decimal(7,2), 22:decimal(7,2)
- Statistics: Num rows: 1 Data size: 1388 Basic
stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 2900 Basic
stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: int), _col1
(type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5
(type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int), _col9
(type: int), _col10 (type: int), _col11 (type: decimal(7,2)), _col12 (type:
decimal(7,2)), _col13 (type: decimal(7,2)), _col14 (type: decimal(7,2)), _col15
(type: decimal(7,2)), _col16 (type: decimal(7,2)), _col17 (type: decimal(7,2)),
_col18 (type: decimal(7,2)), _ [...]
Filter Operator
Filter Vectorization:
@@ -561,7 +524,62 @@ STAGE PLANS:
dataColumns: ss_sold_date_sk:int, ss_sold_time_sk:int,
ss_item_sk2:int, ss_customer_sk2:int, ss_cdemo_sk:int, ss_hdemo_sk:int,
ss_addr_sk:int, ss_store_sk:int, ss_promo_sk:int, ss_ticket_number:int,
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_coup [...]
partitionColumnCount: 0
scratchColumnTypeNames: [double, double, bigint, bigint,
bigint, bigint, bigint, bigint, string, bigint, bigint, bigint, bigint, bigint,
bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(7,2),
decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2),
decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2), decimal(7,2),
decimal(7,2), decimal(7,2), bigint, bigint]
- Reducer 3
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: store_sales
+ filterExpr: ((ss_sold_date_sk = 2451181) and ss_item_sk is
not null and ss_customer_sk is not null) (type: boolean)
+ Statistics: Num rows: 2 Data size: 35#### Basic stats:
COMPLETE Column stats: COMPLETE
+ TableScan Vectorization:
+ native: true
+ vectorizationSchemaColumns: [0:ss_sold_date_sk:int,
1:ss_sold_time_sk:int, 2:ss_item_sk:int, 3:ss_customer_sk:int,
4:ss_cdemo_sk:int, 5:ss_hdemo_sk:int, 6:ss_addr_sk:int, 7:ss_store_sk:int,
8:ss_promo_sk:int, 9:ss_ticket_number:int, 10:ss_quantity:int,
11:ss_wholesale_cost:decimal(7,2), 12:ss_list_price:decimal(7,2),
13:ss_sales_price:decimal(7,2), 14:ss_ext_discount_amt:decimal(7,2),
15:ss_ext_sales_price:decimal(7,2), 16:ss_ext_wholesale_cost:decimal(7,2),
17:ss_e [...]
+ Filter Operator
+ Filter Vectorization:
+ className: VectorFilterOperator
+ native: true
+ predicateExpression: FilterExprAndExpr(children:
FilterLongColEqualLongScalar(col 0:int, val 2451181), SelectColumnIsNotNull(col
2:int), SelectColumnIsNotNull(col 3:int))
+ predicate: ((ss_sold_date_sk = 2451181) and ss_item_sk is
not null and ss_customer_sk is not null) (type: boolean)
+ Statistics: Num rows: 2 Data size: 1400 Basic stats:
COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: PARTITION__SPEC__ID (type: int),
PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type:
bigint), 2451181 (type: int), ss_sold_time_sk (type: int), ss_item_sk (type:
int), ss_customer_sk (type: int), ss_cdemo_sk (type: int), ss_hdemo_sk (type:
int), ss_addr_sk (type: int), ss_store_sk (type: int), ss_promo_sk (type: int),
ss_ticket_number (type: int), ss_quantity (type: int), ss_wholesale_cost (type:
decimal(7,2)), ss_list_price [...]
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4,
_col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14,
_col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24,
_col25, _col26
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [23, 24, 25, 26, 27, 1,
2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22]
+ selectExpressions: ConstantVectorExpression(val
2451181) -> 27:int
+ Statistics: Num rows: 2 Data size: 1808 Basic stats:
COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col6 (type: int), _col7 (type: int)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col6 (type: int), _col7
(type: int)
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkMultiKeyOperator
+ keyColumns: 2:int, 3:int
+ native: true
+ nativeConditionsMet:
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true,
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+ valueColumns: 23:int, 24:bigint, 25:string,
26:bigint, 27:int, 1:int, 4:int, 5:int, 6:int, 7:int, 8:int, 9:int, 10:int,
11:decimal(7,2), 12:decimal(7,2), 13:decimal(7,2), 14:decimal(7,2),
15:decimal(7,2), 16:decimal(7,2), 17:decimal(7,2), 18:decimal(7,2),
19:decimal(7,2), 20:decimal(7,2), 21:decimal(7,2), 22:decimal(7,2)
+ Statistics: Num rows: 2 Data size: 1808 Basic stats:
COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type:
bigint), _col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5
(type: int), _col8 (type: int), _col9 (type: int), _col10 (type: int), _col11
(type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col15
(type: decimal(7,2)), _col16 (type: decimal(7,2)), _col17 (type: decimal(7,2)),
_col18 (type: decimal(7,2)), _col19 (type: decimal(7,2)), _col20 (type:
decimal(7,2)), _col21 (type: [...]
+ Execution mode: vectorized
+ Map Vectorization:
+ enabled: true
+ enabledConditionsMet:
hive.vectorized.use.vectorized.input.format IS true
+ inputFormatFeatureSupport: []
+ featureSupportInUse: []
+ inputFileFormats:
org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+ allNative: true
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 23
+ includeColumns: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12,
13, 14, 15, 16, 17, 18, 19, 20, 21, 22]
+ dataColumns: ss_sold_date_sk:int, ss_sold_time_sk:int,
ss_item_sk:int, 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:int,
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 [...]
+ neededVirtualColumns: #Masked#
+ partitionColumnCount: 0
+ scratchColumnTypeNames: [bigint]
+ Reducer 2
Execution mode: vectorized
Reduce Vectorization:
enabled: true
@@ -597,7 +615,7 @@ STAGE PLANS:
output format:
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
name: default.store_sales
- Reducer 4
+ Reducer 3
Execution mode: vectorized
Reduce Vectorization:
enabled: true
@@ -632,7 +650,7 @@ STAGE PLANS:
output format:
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
name: default.store_sales
- Reducer 5
+ Reducer 4
Execution mode: vectorized
Reduce Vectorization:
enabled: true
@@ -661,13 +679,13 @@ STAGE PLANS:
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Statistics: Num rows: 1 Data size: 1388 Basic stats:
COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 3 Data size: 2900 Basic stats:
COMPLETE Column stats: COMPLETE
table:
input format:
org.apache.iceberg.mr.hive.HiveIcebergInputFormat
output format:
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
name: default.store_sales
- Reducer 6
+ Reducer 5
Execution mode: vectorized
Reduce Vectorization:
enabled: true
@@ -766,6 +784,233 @@ POSTHOOK: Input: default@store_sales
POSTHOOK: Output: hdfs://### HDFS PATH ###
2451181 NULL 1501 1 NULL NULL NULL NULL NULL
NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL
NULL NULL NULL NULL
2451181 NULL 1502 2 NULL NULL NULL NULL NULL
NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL
NULL NULL NULL NULL
+PREHOOK: query: explain
+MERGE INTO store_sales t
+ USING ssv s
+ON (t.ss_item_sk = s.ss_item_sk2
+ AND t.ss_customer_sk=s.ss_customer_sk2
+ AND t.ss_sold_date_sk = "2451181"
+ AND ((Floor((s.ss_item_sk2) / 1000) * 1000) BETWEEN 1000 AND 2000)
+ AND s.ss_ext_discount_amt < 0.0) WHEN matched
+ AND t.ss_ext_discount_amt IS NULL
+THEN UPDATE
+ SET ss_ext_discount_amt = 0.0
+WHEN NOT matched THEN
+ INSERT (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)
+ VALUES (
+ s.ss_sold_time_sk,
+ s.ss_item_sk2,
+ s.ss_customer_sk2,
+ s.ss_cdemo_sk,
+ s.ss_hdemo_sk,
+ s.ss_addr_sk,
+ s.ss_store_sk,
+ s.ss_promo_sk,
+ s.ss_ticket_number,
+ s.ss_quantity,
+ s.ss_wholesale_cost,
+ s.ss_list_price,
+ s.ss_sales_price,
+ s.ss_ext_discount_amt,
+ s.ss_ext_sales_price,
+ s.ss_ext_wholesale_cost,
+ s.ss_ext_list_price,
+ s.ss_ext_tax,
+ s.ss_coupon_amt,
+ s.ss_net_paid,
+ s.ss_net_paid_inc_tax,
+ s.ss_net_profit,
+ "2451181"
+ )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ssv
+PREHOOK: Input: default@store_sales
+PREHOOK: Output: default@merge_tmp_table
+PREHOOK: Output: default@store_sales
+PREHOOK: Output: default@store_sales
+POSTHOOK: query: explain
+MERGE INTO store_sales t
+ USING ssv s
+ON (t.ss_item_sk = s.ss_item_sk2
+ AND t.ss_customer_sk=s.ss_customer_sk2
+ AND t.ss_sold_date_sk = "2451181"
+ AND ((Floor((s.ss_item_sk2) / 1000) * 1000) BETWEEN 1000 AND 2000)
+ AND s.ss_ext_discount_amt < 0.0) WHEN matched
+ AND t.ss_ext_discount_amt IS NULL
+THEN UPDATE
+ SET ss_ext_discount_amt = 0.0
+WHEN NOT matched THEN
+ INSERT (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)
+ VALUES (
+ s.ss_sold_time_sk,
+ s.ss_item_sk2,
+ s.ss_customer_sk2,
+ s.ss_cdemo_sk,
+ s.ss_hdemo_sk,
+ s.ss_addr_sk,
+ s.ss_store_sk,
+ s.ss_promo_sk,
+ s.ss_ticket_number,
+ s.ss_quantity,
+ s.ss_wholesale_cost,
+ s.ss_list_price,
+ s.ss_sales_price,
+ s.ss_ext_discount_amt,
+ s.ss_ext_sales_price,
+ s.ss_ext_wholesale_cost,
+ s.ss_ext_list_price,
+ s.ss_ext_tax,
+ s.ss_coupon_amt,
+ s.ss_net_paid,
+ s.ss_net_paid_inc_tax,
+ s.ss_net_profit,
+ "2451181"
+ )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ssv
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Output: default@merge_tmp_table
+POSTHOOK: Output: default@store_sales
+POSTHOOK: Output: default@store_sales
+Vertex dependency in root stage
+Map 1 <- Map 6 (BROADCAST_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE)
+Reducer 4 <- Map 1 (SIMPLE_EDGE)
+Reducer 5 <- Map 1 (SIMPLE_EDGE)
+
+Stage-6
+ Stats Work{}
+ Stage-0
+ Move Operator
+ table:{"name:":"default.store_sales"}
+ Stage-5
+ Dependency Collection{}
+ Stage-4
+ Reducer 2 vectorized
+ File Output Operator [FS_66]
+ table:{"name:":"default.store_sales"}
+ Select Operator [SEL_65] (rows=1 width=1592)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26"]
+ <-Map 1 [SIMPLE_EDGE] vectorized
+ SHUFFLE [RS_60]
+ Select Operator [SEL_56] (rows=1 width=1476)
+
Output:["_col0","_col1","_col2","_col3","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26"]
+ Filter Operator [FIL_52] (rows=1 width=2976)
+ predicate:((_col23 = _col33) and (_col46 = _col36) and
(_col35 = 2451181) and (floor((_col33 / 1000)) * 1000) BETWEEN 1000 AND 2000
and (_col29 < 0) and _col32 is null)
+ Select Operator [SEL_51] (rows=7 width=950)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43","_col44","_col45","_col46","_col47","_col48"]
+ Map Join Operator [MAPJOIN_50] (rows=7 width=950)
+ Conds:SEL_49._col1, _col2=RS_48._col6, _col7(Left
Outer),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43","_col44","_col45","_col46","_co
[...]
+ <-Map 6 [BROADCAST_EDGE] vectorized
+ BROADCAST [RS_48]
+ PartitionCols:_col6, _col7
+ Select Operator [SEL_47] (rows=2 width=904)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26"]
+ Filter Operator [FIL_46] (rows=2 width=700)
+ predicate:((ss_sold_date_sk = 2451181) and
ss_item_sk is not null and ss_customer_sk is not null)
+ TableScan [TS_2] (rows=2 width=17####)
+
default@store_sales,store_sales,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_date_sk","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"]
+ <-Select Operator [SEL_49] (rows=5 width=380)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
+ TableScan [TS_0] (rows=5 width=372)
+
default@ssv,s,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_sold_time_sk","ss_item_sk2","ss_customer_sk2","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"]
+ Reducer 3 vectorized
+ File Output Operator [FS_68]
+ table:{"name:":"default.store_sales"}
+ Select Operator [SEL_67]
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col3","iceberg_bucket(_col2,
3)"]
+ <-Map 1 [SIMPLE_EDGE] vectorized
+ SHUFFLE [RS_61]
+ PartitionCols:_col3, iceberg_bucket(_col2, 3)
+ Select Operator [SEL_57] (rows=1 width=1388)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22"]
+ Filter Operator [FIL_53] (rows=1 width=2976)
+ predicate:((_col23 = _col33) and (_col46 = _col36) and
(_col35 = 2451181) and (floor((_col33 / 1000)) * 1000) BETWEEN 1000 AND 2000
and (_col29 < 0) and _col32 is null)
+ Please refer to the previous Select Operator [SEL_51]
+ Reducer 4 vectorized
+ File Output Operator [FS_70]
+ table:{"name:":"default.store_sales"}
+ Select Operator [SEL_69]
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col3","iceberg_bucket(_col2,
3)"]
+ <-Map 1 [SIMPLE_EDGE] vectorized
+ SHUFFLE [RS_62]
+ PartitionCols:_col3, iceberg_bucket(_col2, 3)
+ Select Operator [SEL_58] (rows=3 width=966)
+
Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22"]
+ Filter Operator [FIL_54] (rows=3 width=1565)
+ predicate:(_col23 is null and _col46 is null and
_col35 is null)
+ Please refer to the previous Select Operator [SEL_51]
+ Reducer 5 vectorized
+ File Output Operator [FS_74]
+ table:{"name:":"default.merge_tmp_table"}
+ Select Operator [SEL_73] (rows=1 width=4)
+ Output:["_col0"]
+ Filter Operator [FIL_72] (rows=1 width=212)
+ predicate:(_col4 > 1L)
+ Group By Operator [GBY_71] (rows=1 width=212)
+
Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["count(VALUE._col0)"],keys:KEY._col0,
KEY._col1, KEY._col2, KEY._col3
+ <-Map 1 [SIMPLE_EDGE] vectorized
+ SHUFFLE [RS_64]
+ PartitionCols:_col0, _col1, _col2, _col3
+ Group By Operator [GBY_63] (rows=1 width=212)
+
Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["count()"],keys:_col4,
_col39, _col6, _col44
+ Select Operator [SEL_59] (rows=1 width=2976)
+ Output:["_col4","_col6","_col39","_col44"]
+ Filter Operator [FIL_55] (rows=1 width=2976)
+ predicate:((_col23 = _col33) and (_col46 =
_col36) and (_col35 = 2451181) and (floor((_col33 / 1000)) * 1000) BETWEEN 1000
AND 2000 and (_col29 < 0))
+ Please refer to the previous Select Operator
[SEL_51]
+Stage-7
+ Stats Work{}
+ Stage-3
+ Move Operator
+ table:{"name:":"default.merge_tmp_table"}
+ Please refer to the previous Stage-5
+
PREHOOK: query: MERGE INTO store_sales t
USING ssv s
ON (t.ss_item_sk = s.ss_item_sk2