You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "Mostafa Mokhtar (JIRA)" <ji...@apache.org> on 2014/11/06 23:01:35 UTC

[jira] [Created] (HIVE-8765) TPC-DS Q21 : Incorrect join order makes query run slower (Not scaling selectivity by NDV)

Mostafa Mokhtar created HIVE-8765:
-------------------------------------

             Summary: TPC-DS Q21 : Incorrect join order makes query run slower (Not scaling selectivity by NDV)	
                 Key: HIVE-8765
                 URL: https://issues.apache.org/jira/browse/HIVE-8765
             Project: Hive
          Issue Type: Bug
          Components: CBO
    Affects Versions: 0.14.0
            Reporter: Mostafa Mokhtar
            Assignee: Laljo John Pullokkaran
             Fix For: 0.15.0


CBO joins with date_dim first instead of item where item is the more selective join.

Query 
{code}
select  *
 from(select w_warehouse_name
            ,i_item_id
            ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date))
	                then inv_quantity_on_hand 
                      else 0 end) as inv_before
            ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date))
                      then inv_quantity_on_hand 
                      else 0 end) as inv_after
   from inventory
       ,warehouse
       ,item
       ,date_dim
   where i_current_price between 0.99 and 1.49
     and item.i_item_sk          = inventory.inv_item_sk
     and inventory.inv_warehouse_sk   = warehouse.w_warehouse_sk
     and inventory.inv_date_sk    = date_dim.d_date_sk
     and d_date between '1998-03-09' and '1998-05-07'
   group by w_warehouse_name, i_item_id) x
 where (case when inv_before > 0 
             then inv_after / inv_before 
             else null
             end) between 2.0/3.0 and 3.0/2.0
 order by w_warehouse_name
         ,i_item_id
 limit 100
{code}

Logical Plan 
{code}
2014-11-06 16:58:32,041 DEBUG [main]: parse.SemanticAnalyzer (SemanticAnalyzer.java:apply(12631)) - Plan After Join Reordering:
HiveSortRel(fetch=[100]): rowcount = 1.0, cumulative cost = {1.627879384609158E9 rows, 2.0 cpu, 0.0 io}, id = 12521
  HiveSortRel(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC]): rowcount = 1.0, cumulative cost = {1.627879368609158E9 rows, 1.0 cpu, 0.0 io}, id = 12519
    HiveProjectRel(w_warehouse_name=[$0], i_item_id=[$1], inv_before=[$2], inv_after=[$3]): rowcount = 1.0, cumulative cost = {1.627879352609158E9 rows, 0.0 cpu, 0.0 io}, id = 12517
      HiveFilterRel(condition=[between(false, when(>($2, 0), /(CAST($3):DOUBLE, CAST($2):DOUBLE), null), /(2E0, 3E0), /(3E0, 2E0))]): rowcount = 1.0, cumulative cost = {1.627879352609158E9 rows, 0.0 cpu, 0.0 io}, id = 12515
        HiveAggregateRel(group=[{0, 1}], agg#0=[sum($2)], agg#1=[sum($3)]): rowcount = 1.7688372892644288, cumulative cost = {1.627879352609158E9 rows, 0.0 cpu, 0.0 io}, id = 12513
          HiveProjectRel($f0=[$5], $f1=[$7], $f2=[when(<(CAST($10):DATE, CAST('1998-04-08'):DATE), $2, 0)], $f3=[when(>=(CAST($10):DATE, CAST('1998-04-08'):DATE), $2, 0)]): rowcount = 1.8477987480495097, cumulative cost = {1.627879352609158E9 rows, 0.0 cpu, 0.0 io}, id = 12511
            HiveProjectRel(inv_item_sk=[$2], inv_warehouse_sk=[$3], inv_quantity_on_hand=[$4], inv_date_sk=[$5], w_warehouse_sk=[$0], w_warehouse_name=[$1], i_item_sk=[$8], i_item_id=[$9], i_current_price=[$10], d_date_sk=[$6], d_date=[$7]): rowcount = 1.8477987480495097, cumulative cost = {1.627879352609158E9 rows, 0.0 cpu, 0.0 io}, id = 12577
              HiveJoinRel(condition=[=($3, $0)], joinType=[inner]): rowcount = 1.8477987480495097, cumulative cost = {1.627879352609158E9 rows, 0.0 cpu, 0.0 io}, id = 12575
                HiveProjectRel(w_warehouse_sk=[$0], w_warehouse_name=[$2]): rowcount = 27.0, cumulative cost = {0.0 rows, 0.0 cpu, 0.0 io}, id = 12463
                  HiveTableScanRel(table=[[tpcds_bin_partitioned_orc_30000.warehouse]]): rowcount = 27.0, cumulative cost = {0}, id = 12287
                HiveJoinRel(condition=[=($6, $0)], joinType=[inner]): rowcount = 1.8477987480495097, cumulative cost = {1.6278793237613592E9 rows, 0.0 cpu, 0.0 io}, id = 12573
                  HiveJoinRel(condition=[=($3, $4)], joinType=[inner]): rowcount = 22284.45290147709, cumulative cost = {1.627857001E9 rows, 0.0 cpu, 0.0 io}, id = 12534
                    HiveProjectRel(inv_item_sk=[$0], inv_warehouse_sk=[$1], inv_quantity_on_hand=[$2], inv_date_sk=[$3]): rowcount = 1.627857E9, cumulative cost = {0.0 rows, 0.0 cpu, 0.0 io}, id = 12460
                      HiveTableScanRel(table=[[tpcds_bin_partitioned_orc_30000.inventory]]): rowcount = 1.627857E9, cumulative cost = {0}, id = 12284
                    HiveProjectRel(d_date_sk=[$0], d_date=[$2]): rowcount = 1.0, cumulative cost = {0.0 rows, 0.0 cpu, 0.0 io}, id = 12507
                      HiveFilterRel(condition=[between(false, $2, '1998-03-09', '1998-05-07')]): rowcount = 1.0, cumulative cost = {0.0 rows, 0.0 cpu, 0.0 io}, id = 12505
                        HiveTableScanRel(table=[[tpcds_bin_partitioned_orc_30000.date_dim]]): rowcount = 73049.0, cumulative cost = {0}, id = 12286
                  HiveProjectRel(i_item_sk=[$0], i_item_id=[$1], i_current_price=[$5]): rowcount = 38.308457711442784, cumulative cost = {0.0 rows, 0.0 cpu, 0.0 io}, id = 12501
                    HiveFilterRel(condition=[between(false, $5, 9.8999999999999999111E-1, 1.4899999999999999911E0)]): rowcount = 38.308457711442784, cumulative cost = {0.0 rows, 0.0 cpu, 0.0 io}, id = 12499
                      HiveTableScanRel(table=[[tpcds_bin_partitioned_orc_30000.item]]): rowcount = 462000.0, cumulative cost = {0}, id = 12285
{code}


Physical plan
{code}
STAGE DEPENDENCIES:
  Stage-1 is a root stage
  Stage-0 depends on stages: Stage-1

STAGE PLANS:
  Stage: Stage-1
    Tez
      Edges:
        Map 3 <- Map 1 (BROADCAST_EDGE), Map 2 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE)
        Reducer 4 <- Map 3 (SIMPLE_EDGE)
        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
      DagName: mmokhtar_20141104001212_4ebd83eb-0b1a-4375-aa32-b6455db0b8f9:1
      Vertices:
        Map 1 
            Map Operator Tree:
                TableScan
                  alias: warehouse
                  filterExpr: w_warehouse_sk is not null (type: boolean)
                  Statistics: Num rows: 27 Data size: 27802 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: w_warehouse_sk is not null (type: boolean)
                    Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: w_warehouse_sk (type: int), w_warehouse_name (type: string)
                      outputColumnNames: _col0, _col1
                      Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col1 (type: string)
            Execution mode: vectorized
        Map 2 
            Map Operator Tree:
                TableScan
                  alias: item
                  filterExpr: (i_current_price BETWEEN 0.99 AND 1.49 and i_item_sk is not null) (type: boolean)
                  Statistics: Num rows: 462000 Data size: 663862160 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (i_current_price BETWEEN 0.99 AND 1.49 and i_item_sk is not null) (type: boolean)
                    Statistics: Num rows: 231000 Data size: 24945664 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: i_item_sk (type: int), i_item_id (type: string)
                      outputColumnNames: _col0, _col1
                      Statistics: Num rows: 231000 Data size: 24024000 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 231000 Data size: 24024000 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col1 (type: string)
            Execution mode: vectorized
        Map 3 
            Map Operator Tree:
                TableScan
                  alias: inventory
                  filterExpr: (inv_item_sk is not null and inv_warehouse_sk is not null) (type: boolean)
                  Statistics: Num rows: 1627857000 Data size: 19208695084 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (inv_item_sk is not null and inv_warehouse_sk is not null) (type: boolean)
                    Statistics: Num rows: 1627857000 Data size: 25720123084 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: inv_item_sk (type: int), inv_warehouse_sk (type: int), inv_quantity_on_hand (type: int), inv_date_sk (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Statistics: Num rows: 1627857000 Data size: 25720123084 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        condition expressions:
                          0 {_col0} {_col1} {_col2}
                          1 {_col1}
                        keys:
                          0 _col3 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col0, _col1, _col2, _col5
                        input vertices:
                          1 Map 6
                        Statistics: Num rows: 1820114157 Data size: 185651644014 Basic stats: COMPLETE Column stats: COMPLETE
                        Map Join Operator
                          condition map:
                               Inner Join 0 to 1
                          condition expressions:
                            0 {_col1} {_col2} {_col5}
                            1 {_col1}
                          keys:
                            0 _col0 (type: int)
                            1 _col0 (type: int)
                          outputColumnNames: _col1, _col2, _col5, _col7
                          input vertices:
                            1 Map 2
                          Statistics: Num rows: 1913285356 Data size: 378830500488 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: _col1 (type: int), _col2 (type: int), _col5 (type: string), _col7 (type: string)
                            outputColumnNames: _col1, _col2, _col5, _col7
                            Statistics: Num rows: 1913285356 Data size: 378830500488 Basic stats: COMPLETE Column stats: COMPLETE
                            Map Join Operator
                              condition map:
                                   Inner Join 0 to 1
                              condition expressions:
                                0 {_col1}
                                1 {_col2} {_col5} {_col7}
                              keys:
                                0 _col0 (type: int)
                                1 _col1 (type: int)
                              outputColumnNames: _col1, _col4, _col7, _col9
                              input vertices:
                                0 Map 1
                              Statistics: Num rows: 2348122936 Data size: 699740634928 Basic stats: COMPLETE Column stats: COMPLETE
                              Select Operator
                                expressions: _col1 (type: string), _col9 (type: string), CASE WHEN ((CAST( _col7 AS DATE) < 1998-04-08)) THEN (_col4) ELSE (0) END (type: int), CASE WHEN ((CAST( _col7 AS DATE) >= 1998-04-08)) THEN (_col4) ELSE (0) END (type: int)
                                outputColumnNames: _col0, _col1, _col2, _col3
                                Statistics: Num rows: 2348122936 Data size: 699740634928 Basic stats: COMPLETE Column stats: COMPLETE
                                Group By Operator
                                  aggregations: sum(_col2), sum(_col3)
                                  keys: _col0 (type: string), _col1 (type: string)
                                  mode: hash
                                  outputColumnNames: _col0, _col1, _col2, _col3
                                  Statistics: Num rows: 1174061468 Data size: 253597277088 Basic stats: COMPLETE Column stats: COMPLETE
                                  Reduce Output Operator
                                    key expressions: _col0 (type: string), _col1 (type: string)
                                    sort order: ++
                                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                                    Statistics: Num rows: 1174061468 Data size: 253597277088 Basic stats: COMPLETE Column stats: COMPLETE
                                    value expressions: _col2 (type: bigint), _col3 (type: bigint)
        Map 6 
            Map Operator Tree:
                TableScan
                  alias: date_dim
                  filterExpr: (d_date BETWEEN '1998-03-09' AND '1998-05-07' and d_date_sk is not null) (type: boolean)
                  Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (d_date BETWEEN '1998-03-09' AND '1998-05-07' and d_date_sk is not null) (type: boolean)
                    Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: d_date_sk (type: int), d_date (type: string)
                      outputColumnNames: _col0, _col1
                      Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col1 (type: string)
                      Select Operator
                        expressions: _col0 (type: int)
                        outputColumnNames: _col0
                        Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                        Group By Operator
                          keys: _col0 (type: int)
                          mode: hash
                          outputColumnNames: _col0
                          Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                          Dynamic Partitioning Event Operator
                            Target Input: inventory
                            Partition key expr: inv_date_sk
                            Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                            Target column: inv_date_sk
                            Target Vertex: Map 3
            Execution mode: vectorized
        Reducer 4 
            Reduce Operator Tree:
              Group By Operator
                aggregations: sum(VALUE._col0), sum(VALUE._col1)
                keys: KEY._col0 (type: string), KEY._col1 (type: string)
                mode: mergepartial
                outputColumnNames: _col0, _col1, _col2, _col3
                Statistics: Num rows: 3263127 Data size: 704835432 Basic stats: COMPLETE Column stats: COMPLETE
                Filter Operator
                  predicate: CASE WHEN ((_col2 > 0)) THEN ((UDFToDouble(_col3) / UDFToDouble(_col2))) ELSE (null) END BETWEEN 0.6666666666666666 AND 1.5 (type: boolean)
                  Statistics: Num rows: 1631563 Data size: 352417608 Basic stats: COMPLETE Column stats: COMPLETE
                  Select Operator
                    expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint), _col3 (type: bigint)
                    outputColumnNames: _col0, _col1, _col2, _col3
                    Statistics: Num rows: 1631563 Data size: 352417608 Basic stats: COMPLETE Column stats: COMPLETE
                    Reduce Output Operator
                      key expressions: _col0 (type: string), _col1 (type: string)
                      sort order: ++
                      Statistics: Num rows: 1631563 Data size: 352417608 Basic stats: COMPLETE Column stats: COMPLETE
                      TopN Hash Memory Usage: 0.04
                      value expressions: _col2 (type: bigint), _col3 (type: bigint)
        Reducer 5 
            Reduce Operator Tree:
              Select Operator
                expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), VALUE._col0 (type: bigint), VALUE._col1 (type: bigint)
                outputColumnNames: _col0, _col1, _col2, _col3
                Statistics: Num rows: 1631563 Data size: 352417608 Basic stats: COMPLETE Column stats: COMPLETE
                Limit
                  Number of rows: 100
                  Statistics: Num rows: 100 Data size: 21600 Basic stats: COMPLETE Column stats: COMPLETE
                  File Output Operator
                    compressed: false
                    Statistics: Num rows: 100 Data size: 21600 Basic stats: COMPLETE Column stats: COMPLETE
                    table:
                        input format: org.apache.hadoop.mapred.TextInputFormat
                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
            Execution mode: vectorized

  Stage: Stage-0
    Fetch Operator
      limit: 100
      Processor Tree:
        ListSink

Time taken: 6.142 seconds, Fetched: 205 row(s)
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)