[ 
https://issues.apache.org/jira/browse/HIVE-17178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16368436#comment-16368436
 ] 

Hive QA commented on HIVE-17178:
--------------------------------



Here are the results of testing the latest attachment:
https://issues.apache.org/jira/secure/attachment/12911046/HIVE-17178.4.patch

{color:green}SUCCESS:{color} +1 due to 1 test(s) being added or modified.

{color:red}ERROR:{color} -1 due to 33 failed/errored test(s), 13786 tests 
executed
*Failed tests:*
{noformat}
org.apache.hadoop.hive.cli.TestAccumuloCliDriver.testCliDriver[accumulo_queries]
 (batchId=240)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[ppd_join5] (batchId=36)
org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver[row__id] (batchId=78)
org.apache.hadoop.hive.cli.TestEncryptedHDFSCliDriver.testCliDriver[encryption_move_tbl]
 (batchId=174)
org.apache.hadoop.hive.cli.TestMiniDruidCliDriver.testCliDriver[druidmini_mv] 
(batchId=248)
org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver[llap_smb] 
(batchId=151)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[insert_values_orig_table_use_metadata]
 (batchId=166)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[llap_acid] 
(batchId=170)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[llap_acid_fast]
 (batchId=161)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[mergejoin] 
(batchId=166)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[resourceplan]
 (batchId=163)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[results_cache_1]
 (batchId=167)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[sysdb] 
(batchId=160)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[tez_bmj_schema_evolution]
 (batchId=152)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[tez_dynpart_hashjoin_1]
 (batchId=170)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[tez_smb_1] 
(batchId=168)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[tez_smb_main]
 (batchId=158)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[vector_string_decimal]
 (batchId=152)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[vector_udf_string_to_boolean]
 (batchId=164)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[vectorization_div0]
 (batchId=167)
org.apache.hadoop.hive.cli.TestMiniLlapLocalCliDriver.testCliDriver[vectorized_mapjoin3]
 (batchId=154)
org.apache.hadoop.hive.cli.TestSparkCliDriver.testCliDriver[ppd_join5] 
(batchId=121)
org.apache.hadoop.hive.cli.control.TestDanglingQOuts.checkDanglingQOut 
(batchId=221)
org.apache.hadoop.hive.ql.TestAcidOnTez.testGetSplitsLocks (batchId=224)
org.apache.hadoop.hive.ql.parse.TestParseNegativeDriver.testCliDriver[ambiguous_join_col]
 (batchId=247)
org.apache.hive.beeline.cli.TestHiveCli.testNoErrorDB (batchId=187)
org.apache.hive.hcatalog.listener.TestDbNotificationListener.alterIndex 
(batchId=242)
org.apache.hive.hcatalog.listener.TestDbNotificationListener.createIndex 
(batchId=242)
org.apache.hive.hcatalog.listener.TestDbNotificationListener.dropIndex 
(batchId=242)
org.apache.hive.jdbc.TestJdbcWithMiniLlap.testLlapInputFormatEndToEnd 
(batchId=235)
org.apache.hive.jdbc.TestSSL.testConnectionMismatch (batchId=234)
org.apache.hive.jdbc.TestSSL.testConnectionWrongCertCN (batchId=234)
org.apache.hive.jdbc.TestSSL.testMetastoreConnectionWrongCertCN (batchId=234)
{noformat}

Test results: https://builds.apache.org/job/PreCommit-HIVE-Build/9266/testReport
Console output: https://builds.apache.org/job/PreCommit-HIVE-Build/9266/console
Test logs: http://104.198.109.242/logs/PreCommit-HIVE-Build-9266/

Messages:
{noformat}
Executing org.apache.hive.ptest.execution.TestCheckPhase
Executing org.apache.hive.ptest.execution.PrepPhase
Executing org.apache.hive.ptest.execution.YetusPhase
Executing org.apache.hive.ptest.execution.ExecutionPhase
Executing org.apache.hive.ptest.execution.ReportingPhase
Tests exited with: TestsFailedException: 33 tests failed
{noformat}

This message is automatically generated.

ATTACHMENT ID: 12911046 - PreCommit-HIVE-Build

> Spark Partition Pruning Sink Operator can't target multiple Works
> -----------------------------------------------------------------
>
>                 Key: HIVE-17178
>                 URL: https://issues.apache.org/jira/browse/HIVE-17178
>             Project: Hive
>          Issue Type: Sub-task
>          Components: Spark
>            Reporter: Sahil Takiar
>            Assignee: Rui Li
>            Priority: Major
>         Attachments: HIVE-17178.1.patch, HIVE-17178.2.patch, 
> HIVE-17178.3.patch, HIVE-17178.4.patch
>
>
> A Spark Partition Pruning Sink Operator cannot be used to target multiple Map 
> Work objects. The entire DPP subtree (SEL-GBY-SPARKPRUNINGSINK) is duplicated 
> if a single table needs to be used to target multiple Map Works.
> The following query shows the issue:
> {code}
> set hive.spark.dynamic.partition.pruning=true;
> set hive.auto.convert.join=true;
> create table part_table_1 (col int) partitioned by (part_col int);
> create table part_table_2 (col int) partitioned by (part_col int);
> create table regular_table (col int);
> insert into table regular_table values (1);
> alter table part_table_1 add partition (part_col=1);
> insert into table part_table_1 partition (part_col=1) values (1), (2), (3), 
> (4);
> alter table part_table_1 add partition (part_col=2);
> insert into table part_table_1 partition (part_col=2) values (1), (2), (3), 
> (4);
> alter table part_table_2 add partition (part_col=1);
> insert into table part_table_2 partition (part_col=1) values (1), (2), (3), 
> (4);
> alter table part_table_2 add partition (part_col=2);
> insert into table part_table_2 partition (part_col=2) values (1), (2), (3), 
> (4);
> explain select * from regular_table, part_table_1, part_table_2 where 
> regular_table.col = part_table_1.part_col and regular_table.col = 
> part_table_2.part_col;
> {code}
> The explain plan is
> {code}
> STAGE DEPENDENCIES:
>   Stage-2 is a root stage
>   Stage-1 depends on stages: Stage-2
>   Stage-0 depends on stages: Stage-1
> STAGE PLANS:
>   Stage: Stage-2
>     Spark
> #### A masked pattern was here ####
>       Vertices:
>         Map 1 
>             Map Operator Tree:
>                 TableScan
>                   alias: regular_table
>                   Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
> Column stats: NONE
>                   Filter Operator
>                     predicate: col is not null (type: boolean)
>                     Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                     Select Operator
>                       expressions: col (type: int)
>                       outputColumnNames: _col0
>                       Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                       Spark HashTable Sink Operator
>                         keys:
>                           0 _col0 (type: int)
>                           1 _col1 (type: int)
>                           2 _col1 (type: int)
>                       Select Operator
>                         expressions: _col0 (type: int)
>                         outputColumnNames: _col0
>                         Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                         Group By Operator
>                           keys: _col0 (type: int)
>                           mode: hash
>                           outputColumnNames: _col0
>                           Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                           Spark Partition Pruning Sink Operator
>                             partition key expr: part_col
>                             Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                             target column name: part_col
>                             target work: Map 2
>                       Select Operator
>                         expressions: _col0 (type: int)
>                         outputColumnNames: _col0
>                         Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                         Group By Operator
>                           keys: _col0 (type: int)
>                           mode: hash
>                           outputColumnNames: _col0
>                           Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                           Spark Partition Pruning Sink Operator
>                             partition key expr: part_col
>                             Statistics: Num rows: 1 Data size: 1 Basic stats: 
> COMPLETE Column stats: NONE
>                             target column name: part_col
>                             target work: Map 3
>             Local Work:
>               Map Reduce Local Work
>         Map 3 
>             Map Operator Tree:
>                 TableScan
>                   alias: part_table_2
>                   Statistics: Num rows: 8 Data size: 8 Basic stats: COMPLETE 
> Column stats: NONE
>                   Select Operator
>                     expressions: col (type: int), part_col (type: int)
>                     outputColumnNames: _col0, _col1
>                     Statistics: Num rows: 8 Data size: 8 Basic stats: 
> COMPLETE Column stats: NONE
>                     Spark HashTable Sink Operator
>                       keys:
>                         0 _col0 (type: int)
>                         1 _col1 (type: int)
>                         2 _col1 (type: int)
>                     Select Operator
>                       expressions: _col1 (type: int)
>                       outputColumnNames: _col0
>                       Statistics: Num rows: 8 Data size: 8 Basic stats: 
> COMPLETE Column stats: NONE
>                       Group By Operator
>                         keys: _col0 (type: int)
>                         mode: hash
>                         outputColumnNames: _col0
>                         Statistics: Num rows: 8 Data size: 8 Basic stats: 
> COMPLETE Column stats: NONE
>                         Spark Partition Pruning Sink Operator
>                           partition key expr: part_col
>                           Statistics: Num rows: 8 Data size: 8 Basic stats: 
> COMPLETE Column stats: NONE
>                           target column name: part_col
>                           target work: Map 2
>             Local Work:
>               Map Reduce Local Work
>   Stage: Stage-1
>     Spark
> #### A masked pattern was here ####
>       Vertices:
>         Map 2 
>             Map Operator Tree:
>                 TableScan
>                   alias: part_table_1
>                   Statistics: Num rows: 8 Data size: 8 Basic stats: COMPLETE 
> Column stats: NONE
>                   Select Operator
>                     expressions: col (type: int), part_col (type: int)
>                     outputColumnNames: _col0, _col1
>                     Statistics: Num rows: 8 Data size: 8 Basic stats: 
> COMPLETE Column stats: NONE
>                     Map Join Operator
>                       condition map:
>                            Inner Join 0 to 1
>                            Inner Join 0 to 2
>                       keys:
>                         0 _col0 (type: int)
>                         1 _col1 (type: int)
>                         2 _col1 (type: int)
>                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
>                       input vertices:
>                         0 Map 1
>                         2 Map 3
>                       Statistics: Num rows: 17 Data size: 17 Basic stats: 
> COMPLETE Column stats: NONE
>                       File Output Operator
>                         compressed: false
>                         Statistics: Num rows: 17 Data size: 17 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
>             Local Work:
>               Map Reduce Local Work
>   Stage: Stage-0
>     Fetch Operator
>       limit: -1
>       Processor Tree:
>         ListSink
> {code}
> The DPP subtrees on Map 1 are exactly the same. We should be able to combine 
> them, which avoids doing duplicate work.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to