Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 7E4D3200C36 for ; Fri, 24 Feb 2017 02:08:39 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 7CF04160B7E; Fri, 24 Feb 2017 01:08:39 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 4EAD2160B79 for ; Fri, 24 Feb 2017 02:08:38 +0100 (CET) Received: (qmail 34105 invoked by uid 500); 24 Feb 2017 01:08:35 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 31658 invoked by uid 99); 24 Feb 2017 01:08:33 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 24 Feb 2017 01:08:33 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id BFF11DFFC2; Fri, 24 Feb 2017 01:08:33 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sershe@apache.org To: commits@hive.apache.org Date: Fri, 24 Feb 2017 01:08:50 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [18/50] [abbrv] hive git commit: HIVE-15904: select query throwing Null Pointer Exception from org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization.generateSemiJoinOperatorPlan (Jason Dere, reviewed by Gunther Hagleitner) archived-at: Fri, 24 Feb 2017 01:08:39 -0000 HIVE-15904: select query throwing Null Pointer Exception from org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization.generateSemiJoinOperatorPlan (Jason Dere, reviewed by Gunther Hagleitner) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0debf9f2 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0debf9f2 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0debf9f2 Branch: refs/heads/hive-14535 Commit: 0debf9f2916b2ed115e1cdb392a595ae7cf0c761 Parents: 56f6c9d Author: Jason Dere Authored: Sun Feb 19 15:58:25 2017 -0800 Committer: Jason Dere Committed: Sun Feb 19 15:58:25 2017 -0800 ---------------------------------------------------------------------- .../test/resources/testconfiguration.properties | 1 + .../DynamicPartitionPruningOptimization.java | 36 ++- .../dynamic_semijoin_reduction_2.q | 41 +++ .../llap/dynamic_semijoin_reduction_2.q.out | 301 +++++++++++++++++++ 4 files changed, 365 insertions(+), 14 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/itests/src/test/resources/testconfiguration.properties ---------------------------------------------------------------------- diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index 7c54275..4a69bcc 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -482,6 +482,7 @@ minillaplocal.query.files=acid_globallimit.q,\ disable_merge_for_bucketing.q,\ dynamic_partition_pruning.q,\ dynamic_semijoin_reduction.q,\ + dynamic_semijoin_reduction_2.q,\ dynpart_sort_opt_vectorization.q,\ dynpart_sort_optimization.q,\ dynpart_sort_optimization_acid.q,\ http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java index c8691e8..8692c45 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java @@ -398,25 +398,33 @@ public class DynamicPartitionPruningOptimization implements NodeProcessor { String internalColName = null; ExprNodeDesc exprNodeDesc = key; // Find the ExprNodeColumnDesc - while (!(exprNodeDesc instanceof ExprNodeColumnDesc)) { + while (!(exprNodeDesc instanceof ExprNodeColumnDesc) && + (exprNodeDesc.getChildren() != null)) { exprNodeDesc = exprNodeDesc.getChildren().get(0); } - internalColName = ((ExprNodeColumnDesc) exprNodeDesc).getColumn(); - ExprNodeColumnDesc colExpr = ((ExprNodeColumnDesc)(parentOfRS. - getColumnExprMap().get(internalColName))); - String colName = ExprNodeDescUtils.extractColName(colExpr); + if (exprNodeDesc instanceof ExprNodeColumnDesc) { + internalColName = ((ExprNodeColumnDesc) exprNodeDesc).getColumn(); - // Fetch the TableScan Operator. - Operator op = parentOfRS.getParentOperators().get(0); - while (op != null && !(op instanceof TableScanOperator)) { - op = op.getParentOperators().get(0); - } - assert op != null; + ExprNodeColumnDesc colExpr = ((ExprNodeColumnDesc) (parentOfRS. + getColumnExprMap().get(internalColName))); + String colName = ExprNodeDescUtils.extractColName(colExpr); + + // Fetch the TableScan Operator. + Operator op = parentOfRS.getParentOperators().get(0); + while (op != null && !(op instanceof TableScanOperator)) { + op = op.getParentOperators().get(0); + } + assert op != null; - Table table = ((TableScanOperator) op).getConf().getTableMetadata(); - if (table.isPartitionKey(colName)) { - // The column is partition column, skip the optimization. + Table table = ((TableScanOperator) op).getConf().getTableMetadata(); + if (table.isPartitionKey(colName)) { + // The column is partition column, skip the optimization. + return false; + } + } else { + // No column found! + // Bail out return false; } } http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q b/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q new file mode 100644 index 0000000..2306395 --- /dev/null +++ b/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q @@ -0,0 +1,41 @@ +set hive.compute.query.using.stats=false; +set hive.mapred.mode=nonstrict; +set hive.explain.user=false; +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; +set hive.tez.dynamic.partition.pruning=true; +set hive.tez.dynamic.semijoin.reduction=true; +set hive.optimize.metadataonly=false; +set hive.optimize.index.filter=true; + +CREATE TABLE `table_1`( + `bigint_col_7` bigint, + `decimal2016_col_26` decimal(20,16), + `tinyint_col_3` tinyint, + `decimal2612_col_77` decimal(26,12), + `timestamp_col_9` timestamp); + +CREATE TABLE `table_18`( + `tinyint_col_15` tinyint, + `decimal2709_col_9` decimal(27,9), + `tinyint_col_20` tinyint, + `smallint_col_19` smallint, + `decimal1911_col_16` decimal(19,11), + `timestamp_col_18` timestamp); + +-- HIVE-15904 +EXPLAIN +SELECT +COUNT(*) +FROM table_1 t1 +INNER JOIN table_18 t2 ON (((t2.tinyint_col_15) = (t1.bigint_col_7)) AND +((t2.decimal2709_col_9) = (t1.decimal2016_col_26))) AND +((t2.tinyint_col_20) = (t1.tinyint_col_3)) +WHERE (t2.smallint_col_19) IN (SELECT +COALESCE(-92, -994) AS int_col +FROM table_1 tt1 +INNER JOIN table_18 tt2 ON (tt2.decimal1911_col_16) = (tt1.decimal2612_col_77) +WHERE (t1.timestamp_col_9) = (tt2.timestamp_col_18)); + +drop table table_1; +drop table table_18; http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out new file mode 100644 index 0000000..d3e0e39 --- /dev/null +++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out @@ -0,0 +1,301 @@ +PREHOOK: query: CREATE TABLE `table_1`( + `bigint_col_7` bigint, + `decimal2016_col_26` decimal(20,16), + `tinyint_col_3` tinyint, + `decimal2612_col_77` decimal(26,12), + `timestamp_col_9` timestamp) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table_1 +POSTHOOK: query: CREATE TABLE `table_1`( + `bigint_col_7` bigint, + `decimal2016_col_26` decimal(20,16), + `tinyint_col_3` tinyint, + `decimal2612_col_77` decimal(26,12), + `timestamp_col_9` timestamp) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table_1 +PREHOOK: query: CREATE TABLE `table_18`( + `tinyint_col_15` tinyint, + `decimal2709_col_9` decimal(27,9), + `tinyint_col_20` tinyint, + `smallint_col_19` smallint, + `decimal1911_col_16` decimal(19,11), + `timestamp_col_18` timestamp) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@table_18 +POSTHOOK: query: CREATE TABLE `table_18`( + `tinyint_col_15` tinyint, + `decimal2709_col_9` decimal(27,9), + `tinyint_col_20` tinyint, + `smallint_col_19` smallint, + `decimal1911_col_16` decimal(19,11), + `timestamp_col_18` timestamp) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@table_18 +PREHOOK: query: EXPLAIN +SELECT +COUNT(*) +FROM table_1 t1 +INNER JOIN table_18 t2 ON (((t2.tinyint_col_15) = (t1.bigint_col_7)) AND +((t2.decimal2709_col_9) = (t1.decimal2016_col_26))) AND +((t2.tinyint_col_20) = (t1.tinyint_col_3)) +WHERE (t2.smallint_col_19) IN (SELECT +COALESCE(-92, -994) AS int_col +FROM table_1 tt1 +INNER JOIN table_18 tt2 ON (tt2.decimal1911_col_16) = (tt1.decimal2612_col_77) +WHERE (t1.timestamp_col_9) = (tt2.timestamp_col_18)) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT +COUNT(*) +FROM table_1 t1 +INNER JOIN table_18 t2 ON (((t2.tinyint_col_15) = (t1.bigint_col_7)) AND +((t2.decimal2709_col_9) = (t1.decimal2016_col_26))) AND +((t2.tinyint_col_20) = (t1.tinyint_col_3)) +WHERE (t2.smallint_col_19) IN (SELECT +COALESCE(-92, -994) AS int_col +FROM table_1 tt1 +INNER JOIN table_18 tt2 ON (tt2.decimal1911_col_16) = (tt1.decimal2612_col_77) +WHERE (t1.timestamp_col_9) = (tt2.timestamp_col_18)) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 10 <- Reducer 5 (BROADCAST_EDGE) + Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE) + Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE) + Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE) + Reducer 5 <- Reducer 2 (CUSTOM_SIMPLE_EDGE) + Reducer 8 <- Map 10 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE) + Reducer 9 <- Reducer 8 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: t1 + filterExpr: (bigint_col_7 is not null and decimal2016_col_26 is not null and tinyint_col_3 is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: (bigint_col_7 is not null and decimal2016_col_26 is not null and tinyint_col_3 is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: bigint_col_7 (type: bigint), decimal2016_col_26 (type: decimal(20,16)), tinyint_col_3 (type: tinyint), timestamp_col_9 (type: timestamp) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: decimal(34,16)), _col2 (type: tinyint), _col0 (type: bigint) + sort order: +++ + Map-reduce partition columns: _col1 (type: decimal(34,16)), _col2 (type: tinyint), _col0 (type: bigint) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col3 (type: timestamp) + Execution mode: llap + LLAP IO: no inputs + Map 10 + Map Operator Tree: + TableScan + alias: tt2 + filterExpr: ((timestamp_col_18 = timestamp_col_18) and decimal1911_col_16 is not null and timestamp_col_18 BETWEEN DynamicValue(RS_23_t1_timestamp_col_18_min) AND DynamicValue(RS_23_t1_timestamp_col_18_max) and in_bloom_filter(timestamp_col_18, DynamicValue(RS_23_t1_timestamp_col_18_bloom_filter))) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: ((timestamp_col_18 = timestamp_col_18) and decimal1911_col_16 is not null and timestamp_col_18 BETWEEN DynamicValue(RS_23_t1_timestamp_col_18_min) AND DynamicValue(RS_23_t1_timestamp_col_18_max) and in_bloom_filter(timestamp_col_18, DynamicValue(RS_23_t1_timestamp_col_18_bloom_filter))) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: decimal1911_col_16 (type: decimal(19,11)), timestamp_col_18 (type: timestamp) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: decimal(26,12)) + sort order: + + Map-reduce partition columns: _col0 (type: decimal(26,12)) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: timestamp) + Execution mode: llap + LLAP IO: no inputs + Map 6 + Map Operator Tree: + TableScan + alias: t2 + filterExpr: (tinyint_col_15 is not null and decimal2709_col_9 is not null and tinyint_col_20 is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: (tinyint_col_15 is not null and decimal2709_col_9 is not null and tinyint_col_20 is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: tinyint_col_15 (type: tinyint), decimal2709_col_9 (type: decimal(27,9)), tinyint_col_20 (type: tinyint), smallint_col_19 (type: smallint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: decimal(34,16)), _col2 (type: tinyint), UDFToLong(_col0) (type: bigint) + sort order: +++ + Map-reduce partition columns: _col1 (type: decimal(34,16)), _col2 (type: tinyint), UDFToLong(_col0) (type: bigint) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + value expressions: _col3 (type: smallint) + Execution mode: llap + LLAP IO: no inputs + Map 7 + Map Operator Tree: + TableScan + alias: tt1 + filterExpr: decimal2612_col_77 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: decimal2612_col_77 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: decimal2612_col_77 (type: decimal(26,12)) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: decimal(26,12)) + sort order: + + Map-reduce partition columns: _col0 (type: decimal(26,12)) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Execution mode: llap + LLAP IO: no inputs + Reducer 2 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: decimal(34,16)), _col2 (type: tinyint), _col0 (type: bigint) + 1 _col1 (type: decimal(34,16)), _col2 (type: tinyint), UDFToLong(_col0) (type: bigint) + outputColumnNames: _col3, _col7 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: timestamp), UDFToInteger(_col7) (type: int) + sort order: ++ + Map-reduce partition columns: _col3 (type: timestamp), UDFToInteger(_col7) (type: int) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col3 (type: timestamp) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary) + Reducer 3 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: timestamp), UDFToInteger(_col7) (type: int) + 1 _col1 (type: timestamp), -92 (type: int) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) + Reducer 4 + Execution mode: llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Execution mode: llap + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1) + mode: final + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary) + Reducer 8 + Execution mode: llap + Reduce Operator Tree: + Merge Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: decimal(26,12)) + 1 _col0 (type: decimal(26,12)) + outputColumnNames: _col2 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Group By Operator + keys: _col2 (type: timestamp) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: timestamp) + sort order: + + Map-reduce partition columns: _col0 (type: timestamp) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reducer 9 + Execution mode: llap + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: timestamp) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: timestamp) + outputColumnNames: _col1 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col1 (type: timestamp), -92 (type: int) + sort order: ++ + Map-reduce partition columns: _col1 (type: timestamp), -92 (type: int) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: drop table table_1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@table_1 +PREHOOK: Output: default@table_1 +POSTHOOK: query: drop table table_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@table_1 +POSTHOOK: Output: default@table_1 +PREHOOK: query: drop table table_18 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@table_18 +PREHOOK: Output: default@table_18 +POSTHOOK: query: drop table table_18 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@table_18 +POSTHOOK: Output: default@table_18