Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-8306

Map join sizing done by auto.convert.join.noconditionaltask.size doesn't take into account Hash table overhead and results in OOM

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 0.14.0
    • None
    • Physical Optimizer
    • None

    Description

      When hive.auto.convert.join.noconditionaltask = true we check noconditionaltask.size and if the sum of tables sizes in the map join is less than noconditionaltask.size the plan would generate a Map join, the issue with this is that the calculation doesn't take into account the overhead introduced by different HashTable implementation as results if the sum of input sizes is smaller than the noconditionaltask size by a small margin queries will hit OOM.

      TPC-DS query 64 is a good example for this issue as one as non conditional task size is set to 1,280,000,000 while sum of input is 1,012,379,321 which is 20% smaller than the expected size.

      Vertex

         Map 28 <- Map 11 (BROADCAST_EDGE), Map 12 (BROADCAST_EDGE), Map 14 (BROADCAST_EDGE), Map 15 (BROADCAST_EDGE), Map 16 (BROADCAST_EDGE), Map 24 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE), Map 30 (BROADCAST_EDGE), Map 31 (BROADCAST_EDGE), Map 32 (BROADCAST_EDGE), Map 39 (BROADCAST_EDGE), Map 40 (BROADCAST_EDGE), Map 43 (BROADCAST_EDGE), Map 45 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE)
      

      Exception

      , TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.OutOfMemoryError: Java heap space
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:169)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:180)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:172)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:172)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:167)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
      	at java.lang.Thread.run(Thread.java:744)
      Caused by: java.lang.OutOfMemoryError: Java heap space
      	at org.apache.hadoop.hive.serde2.WriteBuffers.nextBufferToWrite(WriteBuffers.java:206)
      	at org.apache.hadoop.hive.serde2.WriteBuffers.write(WriteBuffers.java:182)
      	at org.apache.hadoop.hive.ql.exec.persistence.MapJoinBytesTableContainer$LazyBinaryKvWriter.writeKey(MapJoinBytesTableContainer.java:189)
      	at org.apache.hadoop.hive.ql.exec.persistence.BytesBytesMultiHashMap.put(BytesBytesMultiHashMap.java:200)
      	at org.apache.hadoop.hive.ql.exec.persistence.MapJoinBytesTableContainer.putRow(MapJoinBytesTableContainer.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.HashTableLoader.load(HashTableLoader.java:114)
      	at org.apache.hadoop.hive.ql.exec.MapJoinOperator.loadHashTable(MapJoinOperator.java:184)
      	at org.apache.hadoop.hive.ql.exec.MapJoinOperator.cleanUpInputFileChangedOp(MapJoinOperator.java:210)
      	at org.apache.hadoop.hive.ql.exec.Operator.cleanUpInputFileChanged(Operator.java:1036)
      	at org.apache.hadoop.hive.ql.exec.Operator.cleanUpInputFileChanged(Operator.java:1040)
      	at org.apache.hadoop.hive.ql.exec.Operator.cleanUpInputFileChanged(Operator.java:1040)
      	at org.apache.hadoop.hive.ql.exec.Operator.cleanUpInputFileChanged(Operator.java:1040)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator.process(VectorMapOperator.java:37)
      	at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.processRow(MapRecordProcessor.java:186)
      	at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:164)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:160)
      	... 12 more
      

      Plan

      STAGE PLANS:
        Stage: Stage-1
          Tez
            Edges:
              Map 18 <- Map 1 (BROADCAST_EDGE), Map 13 (BROADCAST_EDGE), Map 2 (BROADCAST_EDGE), Map 25 (BROADCAST_EDGE), Map 27 (BROADCAST_EDGE), Map 29 (BROADCAST_EDGE), Map 3 (BROADCAST_EDGE), Map 35 (BROADCAST_EDGE), Map 36 (BROADCAST_EDGE), Map 37 (BROADCAST_EDGE), Map 38 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE), Map 41 (BROADCAST_EDGE), Map 42 (BROADCAST_EDGE), Map 44 (BROADCAST_EDGE)
              Map 22 <- Map 33 (BROADCAST_EDGE)
              Map 28 <- Map 11 (BROADCAST_EDGE), Map 12 (BROADCAST_EDGE), Map 14 (BROADCAST_EDGE), Map 15 (BROADCAST_EDGE), Map 16 (BROADCAST_EDGE), Map 24 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE), Map 30 (BROADCAST_EDGE), Map 31 (BROADCAST_EDGE), Map 32 (BROADCAST_EDGE), Map 39 (BROADCAST_EDGE), Map 40 (BROADCAST_EDGE), Map 43 (BROADCAST_EDGE), Map 45 (BROADCAST_EDGE), Map 5 (BROADCAST_EDGE)
              Map 6 <- Map 21 (BROADCAST_EDGE)
              Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
              Reducer 19 <- Map 18 (SIMPLE_EDGE), Map 34 (SIMPLE_EDGE), Reducer 23 (SIMPLE_EDGE)
              Reducer 20 <- Reducer 19 (SIMPLE_EDGE)
              Reducer 23 <- Map 22 (SIMPLE_EDGE)
              Reducer 7 <- Map 6 (SIMPLE_EDGE)
              Reducer 8 <- Map 17 (SIMPLE_EDGE), Map 28 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
              Reducer 9 <- Reducer 20 (BROADCAST_EDGE), Reducer 8 (SIMPLE_EDGE)
            DagName: mmokhtar_20140910163939_fc966812-9b9d-47a2-bdad-eb43f336b848:1
            Vertices:
              Map 1 
                  Map Operator Tree:
                      TableScan
                        alias: hd1
                        filterExpr: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                        Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                          Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: hd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: hd_demo_sk (type: int)
                            Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: hd_income_band_sk (type: int)
                  Execution mode: vectorized
              Map 11 
                  Map Operator Tree:
                      TableScan
                        alias: store_sales
                        filterExpr: (((((((ss_item_sk is not null and ss_ticket_number is not null) and ss_customer_sk is not null) and ss_store_sk is not null) and ss_cdemo_sk is not null) and ss_promo_sk is not null) and ss_hdemo_sk is not null) and ss_addr_sk is not null) (type: boolean)
                        Statistics: Num rows: 550076554 Data size: 47370018896 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (((((((ss_item_sk is not null and ss_ticket_number is not null) and ss_customer_sk is not null) and ss_store_sk is not null) and ss_cdemo_sk is not null) and ss_promo_sk is not null) and ss_hdemo_sk is not null) and ss_addr_sk is not null) (type: boolean)
                          Statistics: Num rows: 2148737 Data size: 185039176 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ss_item_sk (type: int), ss_ticket_number (type: int)
                            sort order: ++
                            Map-reduce partition columns: ss_item_sk (type: int), ss_ticket_number (type: int)
                            Statistics: Num rows: 2148737 Data size: 185039176 Basic stats: COMPLETE Column stats: NONE
                            value expressions: 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_wholesale_cost (type: float), ss_list_price (type: float), ss_coupon_amt (type: float), ss_sold_date_sk (type: int)
                  Execution mode: vectorized
              Map 12 
                  Map Operator Tree:
                      TableScan
                        alias: cd1
                        filterExpr: cd_demo_sk is not null (type: boolean)
                        Statistics: Num rows: 1920800 Data size: 718379200 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: cd_demo_sk is not null (type: boolean)
                          Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: cd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: cd_demo_sk (type: int)
                            Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: cd_marital_status (type: string)
                  Execution mode: vectorized
              Map 13 
                  Map Operator Tree:
                      TableScan
                        alias: customer
                        filterExpr: (((((c_customer_sk is not null and c_first_sales_date_sk is not null) and c_first_shipto_date_sk is not null) and c_current_cdemo_sk is not null) and c_current_hdemo_sk is not null) and c_current_addr_sk is not null) (type: boolean)
                        Statistics: Num rows: 1600000 Data size: 1241633212 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (((((c_customer_sk is not null and c_first_sales_date_sk is not null) and c_first_shipto_date_sk is not null) and c_current_cdemo_sk is not null) and c_current_hdemo_sk is not null) and c_current_addr_sk is not null) (type: boolean)
                          Statistics: Num rows: 25000 Data size: 19400518 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: c_customer_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: c_customer_sk (type: int)
                            Statistics: Num rows: 25000 Data size: 19400518 Basic stats: COMPLETE Column stats: NONE
                            value expressions: c_current_cdemo_sk (type: int), c_current_hdemo_sk (type: int), c_current_addr_sk (type: int), c_first_shipto_date_sk (type: int), c_first_sales_date_sk (type: int)
                  Execution mode: vectorized
              Map 14 
                  Map Operator Tree:
                      TableScan
                        alias: cd2
                        filterExpr: cd_demo_sk is not null (type: boolean)
                        Statistics: Num rows: 1920800 Data size: 718379200 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: cd_demo_sk is not null (type: boolean)
                          Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: cd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: cd_demo_sk (type: int)
                            Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: cd_marital_status (type: string)
                  Execution mode: vectorized
              Map 15 
                  Map Operator Tree:
                      TableScan
                        alias: hd2
                        filterExpr: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                        Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                          Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: hd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: hd_demo_sk (type: int)
                            Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: hd_income_band_sk (type: int)
                  Execution mode: vectorized
              Map 16 
                  Map Operator Tree:
                      TableScan
                        alias: hd1
                        filterExpr: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                        Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                          Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: hd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: hd_demo_sk (type: int)
                            Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: hd_income_band_sk (type: int)
                  Execution mode: vectorized
              Map 17 
                  Map Operator Tree:
                      TableScan
                        alias: item
                        filterExpr: (((i_item_sk is not null and (i_color) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate')) and i_current_price BETWEEN 35 AND 45) and i_current_price BETWEEN 36 AND 50) (type: boolean)
                        Statistics: Num rows: 48000 Data size: 68732712 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (((i_item_sk is not null and (i_color) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate')) and i_current_price BETWEEN 35 AND 45) and i_current_price BETWEEN 36 AND 50) (type: boolean)
                          Statistics: Num rows: 3000 Data size: 4295794 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: i_item_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: i_item_sk (type: int)
                            Statistics: Num rows: 3000 Data size: 4295794 Basic stats: COMPLETE Column stats: NONE
                            value expressions: i_current_price (type: float), i_color (type: string), i_product_name (type: string)
                  Execution mode: vectorized
              Map 18 
                  Map Operator Tree:
                      TableScan
                        alias: store_returns
                        filterExpr: (sr_item_sk is not null and sr_ticket_number is not null) (type: boolean)
                        Statistics: Num rows: 55578005 Data size: 4155315616 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (sr_item_sk is not null and sr_ticket_number is not null) (type: boolean)
                          Statistics: Num rows: 13894502 Data size: 1038828960 Basic stats: COMPLETE Column stats: NONE
                          Map Join Operator
                            condition map:
                                 Inner Join 0 to 1
                            condition expressions:
                              0 {ss_item_sk} {ss_customer_sk} {ss_cdemo_sk} {ss_hdemo_sk} {ss_addr_sk} {ss_store_sk} {ss_promo_sk} {ss_wholesale_cost} {ss_list_price} {ss_coupon_amt} {ss_sold_date_sk}
                              1 
                            keys:
                              0 ss_item_sk (type: int), ss_ticket_number (type: int)
                              1 sr_item_sk (type: int), sr_ticket_number (type: int)
                            outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col22
                            input vertices:
                              0 Map 4
                            Statistics: Num rows: 15283953 Data size: 1142711808 Basic stats: COMPLETE Column stats: NONE
                            Map Join Operator
                              condition map:
                                   Inner Join 0 to 1
                              condition expressions:
                                0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col22}
                                1 {c_current_cdemo_sk} {c_current_hdemo_sk} {c_current_addr_sk} {c_first_shipto_date_sk} {c_first_sales_date_sk}
                              keys:
                                0 _col2 (type: int)
                                1 c_customer_sk (type: int)
                              outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col22, _col51, _col52, _col53, _col54, _col55
                              input vertices:
                                1 Map 13
                              Statistics: Num rows: 16812348 Data size: 1256983040 Basic stats: COMPLETE Column stats: NONE
                              Map Join Operator
                                condition map:
                                     Inner Join 0 to 1
                                condition expressions:
                                  0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col54} {_col55}
                                  1 
                                keys:
                                  0 _col22 (type: int)
                                  1 d_date_sk (type: int)
                                outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col54, _col55
                                input vertices:
                                  1 Map 44
                                Statistics: Num rows: 18493584 Data size: 1382681344 Basic stats: COMPLETE Column stats: NONE
                                Map Join Operator
                                  condition map:
                                       Inner Join 0 to 1
                                  condition expressions:
                                    0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col54}
                                    1 {d_year}
                                  keys:
                                    0 _col55 (type: int)
                                    1 d_date_sk (type: int)
                                  outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col54, _col107
                                  input vertices:
                                    1 Map 41
                                  Statistics: Num rows: 20342942 Data size: 1520949504 Basic stats: COMPLETE Column stats: NONE
                                  Map Join Operator
                                    condition map:
                                         Inner Join 0 to 1
                                    condition expressions:
                                      0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col107}
                                      1 {d_year}
                                    keys:
                                      0 _col54 (type: int)
                                      1 d_date_sk (type: int)
                                    outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col107, _col138
                                    input vertices:
                                      1 Map 42
                                    Statistics: Num rows: 22377236 Data size: 1673044480 Basic stats: COMPLETE Column stats: NONE
                                    Map Join Operator
                                      condition map:
                                           Inner Join 0 to 1
                                      condition expressions:
                                        0 {_col1} {_col3} {_col4} {_col5} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col107} {_col138}
                                        1 {s_store_name} {s_zip}
                                      keys:
                                        0 _col6 (type: int)
                                        1 s_store_sk (type: int)
                                      outputColumnNames: _col1, _col3, _col4, _col5, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col107, _col138, _col168, _col188
                                      input vertices:
                                        1 Map 29
                                      Statistics: Num rows: 24614960 Data size: 1840348928 Basic stats: COMPLETE Column stats: NONE
                                      Map Join Operator
                                        condition map:
                                             Inner Join 0 to 1
                                        condition expressions:
                                          0 {_col1} {_col4} {_col5} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188}
                                          1 {cd_marital_status}
                                        keys:
                                          0 _col3 (type: int)
                                          1 cd_demo_sk (type: int)
                                        outputColumnNames: _col1, _col4, _col5, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col107, _col138, _col168, _col188, _col197
                                        input vertices:
                                          1 Map 35
                                        Statistics: Num rows: 27076456 Data size: 2024383872 Basic stats: COMPLETE Column stats: NONE
                                        Map Join Operator
                                          condition map:
                                               Inner Join 0 to 1
                                          condition expressions:
                                            0 {_col1} {_col4} {_col5} {_col7} {_col10} {_col11} {_col18} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197}
                                            1 {cd_marital_status}
                                          keys:
                                            0 _col51 (type: int)
                                            1 cd_demo_sk (type: int)
                                          outputColumnNames: _col1, _col4, _col5, _col7, _col10, _col11, _col18, _col52, _col53, _col107, _col138, _col168, _col188, _col197, _col209
                                          input vertices:
                                            1 Map 38
                                          Statistics: Num rows: 29784102 Data size: 2226822400 Basic stats: COMPLETE Column stats: NONE
                                          Map Join Operator
                                            condition map:
                                                 Inner Join 0 to 1
                                            condition expressions:
                                              0 {_col1} {_col4} {_col5} {_col10} {_col11} {_col18} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209}
                                              1 
                                            keys:
                                              0 _col7 (type: int)
                                              1 p_promo_sk (type: int)
                                            outputColumnNames: _col1, _col4, _col5, _col10, _col11, _col18, _col52, _col53, _col107, _col138, _col168, _col188, _col197, _col209
                                            input vertices:
                                              1 Map 3
                                            Statistics: Num rows: 32762512 Data size: 2449504768 Basic stats: COMPLETE Column stats: NONE
                                            Map Join Operator
                                              condition map:
                                                   Inner Join 0 to 1
                                              condition expressions:
                                                0 {_col1} {_col5} {_col10} {_col11} {_col18} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209}
                                                1 {hd_income_band_sk}
                                              keys:
                                                0 _col4 (type: int)
                                                1 hd_demo_sk (type: int)
                                              outputColumnNames: _col1, _col5, _col10, _col11, _col18, _col52, _col53, _col107, _col138, _col168, _col188, _col197, _col209, _col242
                                              input vertices:
                                                1 Map 1
                                              Statistics: Num rows: 36038764 Data size: 2694455296 Basic stats: COMPLETE Column stats: NONE
                                              Map Join Operator
                                                condition map:
                                                     Inner Join 0 to 1
                                                condition expressions:
                                                  0 {_col1} {_col5} {_col10} {_col11} {_col18} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col242}
                                                  1 {hd_income_band_sk}
                                                keys:
                                                  0 _col52 (type: int)
                                                  1 hd_demo_sk (type: int)
                                                outputColumnNames: _col1, _col5, _col10, _col11, _col18, _col53, _col107, _col138, _col168, _col188, _col197, _col209, _col242, _col250
                                                input vertices:
                                                  1 Map 2
                                                Statistics: Num rows: 39642640 Data size: 2963900928 Basic stats: COMPLETE Column stats: NONE
                                                Map Join Operator
                                                  condition map:
                                                       Inner Join 0 to 1
                                                  condition expressions:
                                                    0 {_col1} {_col10} {_col11} {_col18} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col242} {_col250}
                                                    1 {ca_street_number} {ca_street_name} {ca_city} {ca_zip}
                                                  keys:
                                                    0 _col5 (type: int)
                                                    1 ca_address_sk (type: int)
                                                  outputColumnNames: _col1, _col10, _col11, _col18, _col53, _col107, _col138, _col168, _col188, _col197, _col209, _col242, _col250, _col259, _col260, _col263, _col266
                                                  input vertices:
                                                    1 Map 27
                                                  Statistics: Num rows: 43606904 Data size: 3260291072 Basic stats: COMPLETE Column stats: NONE
                                                  Map Join Operator
                                                    condition map:
                                                         Inner Join 0 to 1
                                                    condition expressions:
                                                      0 {_col1} {_col10} {_col11} {_col18} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col242} {_col250} {_col259} {_col260} {_col263} {_col266}
                                                      1 {ca_street_number} {ca_street_name} {ca_city} {ca_zip}
                                                    keys:
                                                      0 _col53 (type: int)
                                                      1 ca_address_sk (type: int)
                                                    outputColumnNames: _col1, _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col242, _col250, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282
                                                    input vertices:
                                                      1 Map 25
                                                    Statistics: Num rows: 47967596 Data size: 3586320384 Basic stats: COMPLETE Column stats: NONE
                                                    Map Join Operator
                                                      condition map:
                                                           Inner Join 0 to 1
                                                      condition expressions:
                                                        0 {_col1} {_col10} {_col11} {_col18} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col250} {_col259} {_col260} {_col263} {_col266} {_col275} {_col276} {_col279} {_col282}
                                                        1 
                                                      keys:
                                                        0 _col242 (type: int)
                                                        1 ib_income_band_sk (type: int)
                                                      outputColumnNames: _col1, _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col250, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282
                                                      input vertices:
                                                        1 Map 36
                                                      Statistics: Num rows: 52764356 Data size: 3944952576 Basic stats: COMPLETE Column stats: NONE
                                                      Map Join Operator
                                                        condition map:
                                                             Inner Join 0 to 1
                                                        condition expressions:
                                                          0 {_col1} {_col10} {_col11} {_col18} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col259} {_col260} {_col263} {_col266} {_col275} {_col276} {_col279} {_col282}
                                                          1 
                                                        keys:
                                                          0 _col250 (type: int)
                                                          1 ib_income_band_sk (type: int)
                                                        outputColumnNames: _col1, _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282
                                                        input vertices:
                                                          1 Map 37
                                                        Statistics: Num rows: 58040792 Data size: 4339447808 Basic stats: COMPLETE Column stats: NONE
                                                        Reduce Output Operator
                                                          key expressions: _col1 (type: int)
                                                          sort order: +
                                                          Map-reduce partition columns: _col1 (type: int)
                                                          Statistics: Num rows: 58040792 Data size: 4339447808 Basic stats: COMPLETE Column stats: NONE
                                                          value expressions: _col10 (type: float), _col11 (type: float), _col18 (type: float), _col107 (type: int), _col138 (type: int), _col168 (type: string), _col188 (type: string), _col197 (type: string), _col209 (type: string), _col259 (type: string), _col260 (type: string), _col263 (type: string), _col266 (type: string), _col275 (type: string), _col276 (type: string), _col279 (type: string), _col282 (type: string)
                  Execution mode: vectorized
              Map 2 
                  Map Operator Tree:
                      TableScan
                        alias: hd2
                        filterExpr: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                        Statistics: Num rows: 7200 Data size: 770400 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (hd_demo_sk is not null and hd_income_band_sk is not null) (type: boolean)
                          Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: hd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: hd_demo_sk (type: int)
                            Statistics: Num rows: 1800 Data size: 192600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: hd_income_band_sk (type: int)
                  Execution mode: vectorized
              Map 21 
                  Map Operator Tree:
                      TableScan
                        alias: catalog_returns
                        filterExpr: (cr_item_sk is not null and cr_order_number is not null) (type: boolean)
                        Statistics: Num rows: 28798881 Data size: 2942039156 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (cr_item_sk is not null and cr_order_number is not null) (type: boolean)
                          Statistics: Num rows: 7199721 Data size: 735509865 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: cr_item_sk (type: int), cr_order_number (type: int)
                            sort order: ++
                            Map-reduce partition columns: cr_item_sk (type: int), cr_order_number (type: int)
                            Statistics: Num rows: 7199721 Data size: 735509865 Basic stats: COMPLETE Column stats: NONE
                            value expressions: cr_refunded_cash (type: float), cr_reversed_charge (type: float), cr_store_credit (type: float)
                  Execution mode: vectorized
              Map 22 
                  Map Operator Tree:
                      TableScan
                        alias: catalog_sales
                        filterExpr: (cs_item_sk is not null and cs_order_number is not null) (type: boolean)
                        Statistics: Num rows: 286549727 Data size: 37743959324 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (cs_item_sk is not null and cs_order_number is not null) (type: boolean)
                          Statistics: Num rows: 71637432 Data size: 9435989863 Basic stats: COMPLETE Column stats: NONE
                          Map Join Operator
                            condition map:
                                 Inner Join 0 to 1
                            condition expressions:
                              0 {cs_item_sk} {cs_ext_list_price}
                              1 {cr_refunded_cash} {cr_reversed_charge} {cr_store_credit}
                            keys:
                              0 cs_item_sk (type: int), cs_order_number (type: int)
                              1 cr_item_sk (type: int), cr_order_number (type: int)
                            outputColumnNames: _col14, _col24, _col59, _col60, _col61
                            input vertices:
                              1 Map 33
                            Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                            Select Operator
                              expressions: _col14 (type: int), _col24 (type: float), _col59 (type: float), _col60 (type: float), _col61 (type: float)
                              outputColumnNames: _col14, _col24, _col59, _col60, _col61
                              Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                              Group By Operator
                                aggregations: sum(_col24), sum(((_col59 + _col60) + _col61))
                                keys: _col14 (type: int)
                                mode: hash
                                outputColumnNames: _col0, _col1, _col2
                                Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                                Reduce Output Operator
                                  key expressions: _col0 (type: int)
                                  sort order: +
                                  Map-reduce partition columns: _col0 (type: int)
                                  Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                                  value expressions: _col1 (type: double), _col2 (type: double)
                  Execution mode: vectorized
              Map 24 
                  Map Operator Tree:
                      TableScan
                        alias: d3
                        filterExpr: d_date_sk is not null (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: d_date_sk is not null (type: boolean)
                          Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: d_date_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: d_date_sk (type: int)
                            Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                            value expressions: d_year (type: int)
                  Execution mode: vectorized
              Map 25 
                  Map Operator Tree:
                      TableScan
                        alias: ad2
                        filterExpr: ca_address_sk is not null (type: boolean)
                        Statistics: Num rows: 800000 Data size: 811903688 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ca_address_sk is not null (type: boolean)
                          Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ca_address_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ca_address_sk (type: int)
                            Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                            value expressions: ca_street_number (type: string), ca_street_name (type: string), ca_city (type: string), ca_zip (type: string)
                  Execution mode: vectorized
              Map 26 
                  Map Operator Tree:
                      TableScan
                        alias: customer
                        filterExpr: (((((c_customer_sk is not null and c_first_sales_date_sk is not null) and c_first_shipto_date_sk is not null) and c_current_cdemo_sk is not null) and c_current_hdemo_sk is not null) and c_current_addr_sk is not null) (type: boolean)
                        Statistics: Num rows: 1600000 Data size: 1241633212 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (((((c_customer_sk is not null and c_first_sales_date_sk is not null) and c_first_shipto_date_sk is not null) and c_current_cdemo_sk is not null) and c_current_hdemo_sk is not null) and c_current_addr_sk is not null) (type: boolean)
                          Statistics: Num rows: 25000 Data size: 19400518 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: c_customer_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: c_customer_sk (type: int)
                            Statistics: Num rows: 25000 Data size: 19400518 Basic stats: COMPLETE Column stats: NONE
                            value expressions: c_current_cdemo_sk (type: int), c_current_hdemo_sk (type: int), c_current_addr_sk (type: int), c_first_shipto_date_sk (type: int), c_first_sales_date_sk (type: int)
                  Execution mode: vectorized
              Map 27 
                  Map Operator Tree:
                      TableScan
                        alias: ad1
                        filterExpr: ca_address_sk is not null (type: boolean)
                        Statistics: Num rows: 800000 Data size: 811903688 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ca_address_sk is not null (type: boolean)
                          Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ca_address_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ca_address_sk (type: int)
                            Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                            value expressions: ca_street_number (type: string), ca_street_name (type: string), ca_city (type: string), ca_zip (type: string)
                  Execution mode: vectorized
              Map 28 
                  Map Operator Tree:
                      TableScan
                        alias: store_returns
                        filterExpr: (sr_item_sk is not null and sr_ticket_number is not null) (type: boolean)
                        Statistics: Num rows: 55578005 Data size: 4155315616 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (sr_item_sk is not null and sr_ticket_number is not null) (type: boolean)
                          Statistics: Num rows: 13894502 Data size: 1038828960 Basic stats: COMPLETE Column stats: NONE
                          Map Join Operator
                            condition map:
                                 Inner Join 0 to 1
                            condition expressions:
                              0 {ss_item_sk} {ss_customer_sk} {ss_cdemo_sk} {ss_hdemo_sk} {ss_addr_sk} {ss_store_sk} {ss_promo_sk} {ss_wholesale_cost} {ss_list_price} {ss_coupon_amt} {ss_sold_date_sk}
                              1 
                            keys:
                              0 ss_item_sk (type: int), ss_ticket_number (type: int)
                              1 sr_item_sk (type: int), sr_ticket_number (type: int)
                            outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col22
                            input vertices:
                              0 Map 11
                            Statistics: Num rows: 15283953 Data size: 1142711808 Basic stats: COMPLETE Column stats: NONE
                            Map Join Operator
                              condition map:
                                   Inner Join 0 to 1
                              condition expressions:
                                0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col22}
                                1 {c_current_cdemo_sk} {c_current_hdemo_sk} {c_current_addr_sk} {c_first_shipto_date_sk} {c_first_sales_date_sk}
                              keys:
                                0 _col2 (type: int)
                                1 c_customer_sk (type: int)
                              outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col22, _col51, _col52, _col53, _col54, _col55
                              input vertices:
                                1 Map 26
                              Statistics: Num rows: 16812348 Data size: 1256983040 Basic stats: COMPLETE Column stats: NONE
                              Map Join Operator
                                condition map:
                                     Inner Join 0 to 1
                                condition expressions:
                                  0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col54} {_col55}
                                  1 
                                keys:
                                  0 _col22 (type: int)
                                  1 d_date_sk (type: int)
                                outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col54, _col55
                                input vertices:
                                  1 Map 30
                                Statistics: Num rows: 18493584 Data size: 1382681344 Basic stats: COMPLETE Column stats: NONE
                                Map Join Operator
                                  condition map:
                                       Inner Join 0 to 1
                                  condition expressions:
                                    0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col54}
                                    1 {d_year}
                                  keys:
                                    0 _col55 (type: int)
                                    1 d_date_sk (type: int)
                                  outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col54, _col107
                                  input vertices:
                                    1 Map 31
                                  Statistics: Num rows: 20342942 Data size: 1520949504 Basic stats: COMPLETE Column stats: NONE
                                  Map Join Operator
                                    condition map:
                                         Inner Join 0 to 1
                                    condition expressions:
                                      0 {_col1} {_col3} {_col4} {_col5} {_col6} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col107}
                                      1 {d_year}
                                    keys:
                                      0 _col54 (type: int)
                                      1 d_date_sk (type: int)
                                    outputColumnNames: _col1, _col3, _col4, _col5, _col6, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col107, _col138
                                    input vertices:
                                      1 Map 24
                                    Statistics: Num rows: 22377236 Data size: 1673044480 Basic stats: COMPLETE Column stats: NONE
                                    Map Join Operator
                                      condition map:
                                           Inner Join 0 to 1
                                      condition expressions:
                                        0 {_col1} {_col3} {_col4} {_col5} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col107} {_col138}
                                        1 {s_store_name} {s_zip}
                                      keys:
                                        0 _col6 (type: int)
                                        1 s_store_sk (type: int)
                                      outputColumnNames: _col1, _col3, _col4, _col5, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col107, _col138, _col168, _col188
                                      input vertices:
                                        1 Map 45
                                      Statistics: Num rows: 24614960 Data size: 1840348928 Basic stats: COMPLETE Column stats: NONE
                                      Map Join Operator
                                        condition map:
                                             Inner Join 0 to 1
                                        condition expressions:
                                          0 {_col1} {_col4} {_col5} {_col7} {_col10} {_col11} {_col18} {_col51} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188}
                                          1 {cd_marital_status}
                                        keys:
                                          0 _col3 (type: int)
                                          1 cd_demo_sk (type: int)
                                        outputColumnNames: _col1, _col4, _col5, _col7, _col10, _col11, _col18, _col51, _col52, _col53, _col107, _col138, _col168, _col188, _col197
                                        input vertices:
                                          1 Map 12
                                        Statistics: Num rows: 27076456 Data size: 2024383872 Basic stats: COMPLETE Column stats: NONE
                                        Map Join Operator
                                          condition map:
                                               Inner Join 0 to 1
                                          condition expressions:
                                            0 {_col1} {_col4} {_col5} {_col7} {_col10} {_col11} {_col18} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197}
                                            1 {cd_marital_status}
                                          keys:
                                            0 _col51 (type: int)
                                            1 cd_demo_sk (type: int)
                                          outputColumnNames: _col1, _col4, _col5, _col7, _col10, _col11, _col18, _col52, _col53, _col107, _col138, _col168, _col188, _col197, _col209
                                          input vertices:
                                            1 Map 14
                                          Statistics: Num rows: 29784102 Data size: 2226822400 Basic stats: COMPLETE Column stats: NONE
                                          Map Join Operator
                                            condition map:
                                                 Inner Join 0 to 1
                                            condition expressions:
                                              0 {_col1} {_col4} {_col5} {_col10} {_col11} {_col18} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209}
                                              1 
                                            keys:
                                              0 _col7 (type: int)
                                              1 p_promo_sk (type: int)
                                            outputColumnNames: _col1, _col4, _col5, _col10, _col11, _col18, _col52, _col53, _col107, _col138, _col168, _col188, _col197, _col209
                                            input vertices:
                                              1 Map 40
                                            Statistics: Num rows: 32762512 Data size: 2449504768 Basic stats: COMPLETE Column stats: NONE
                                            Map Join Operator
                                              condition map:
                                                   Inner Join 0 to 1
                                              condition expressions:
                                                0 {_col1} {_col5} {_col10} {_col11} {_col18} {_col52} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209}
                                                1 {hd_income_band_sk}
                                              keys:
                                                0 _col4 (type: int)
                                                1 hd_demo_sk (type: int)
                                              outputColumnNames: _col1, _col5, _col10, _col11, _col18, _col52, _col53, _col107, _col138, _col168, _col188, _col197, _col209, _col242
                                              input vertices:
                                                1 Map 16
                                              Statistics: Num rows: 36038764 Data size: 2694455296 Basic stats: COMPLETE Column stats: NONE
                                              Map Join Operator
                                                condition map:
                                                     Inner Join 0 to 1
                                                condition expressions:
                                                  0 {_col1} {_col5} {_col10} {_col11} {_col18} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col242}
                                                  1 {hd_income_band_sk}
                                                keys:
                                                  0 _col52 (type: int)
                                                  1 hd_demo_sk (type: int)
                                                outputColumnNames: _col1, _col5, _col10, _col11, _col18, _col53, _col107, _col138, _col168, _col188, _col197, _col209, _col242, _col250
                                                input vertices:
                                                  1 Map 15
                                                Statistics: Num rows: 39642640 Data size: 2963900928 Basic stats: COMPLETE Column stats: NONE
                                                Map Join Operator
                                                  condition map:
                                                       Inner Join 0 to 1
                                                  condition expressions:
                                                    0 {_col1} {_col10} {_col11} {_col18} {_col53} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col242} {_col250}
                                                    1 {ca_street_number} {ca_street_name} {ca_city} {ca_zip}
                                                  keys:
                                                    0 _col5 (type: int)
                                                    1 ca_address_sk (type: int)
                                                  outputColumnNames: _col1, _col10, _col11, _col18, _col53, _col107, _col138, _col168, _col188, _col197, _col209, _col242, _col250, _col259, _col260, _col263, _col266
                                                  input vertices:
                                                    1 Map 39
                                                  Statistics: Num rows: 43606904 Data size: 3260291072 Basic stats: COMPLETE Column stats: NONE
                                                  Map Join Operator
                                                    condition map:
                                                         Inner Join 0 to 1
                                                    condition expressions:
                                                      0 {_col1} {_col10} {_col11} {_col18} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col242} {_col250} {_col259} {_col260} {_col263} {_col266}
                                                      1 {ca_street_number} {ca_street_name} {ca_city} {ca_zip}
                                                    keys:
                                                      0 _col53 (type: int)
                                                      1 ca_address_sk (type: int)
                                                    outputColumnNames: _col1, _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col242, _col250, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282
                                                    input vertices:
                                                      1 Map 43
                                                    Statistics: Num rows: 47967596 Data size: 3586320384 Basic stats: COMPLETE Column stats: NONE
                                                    Map Join Operator
                                                      condition map:
                                                           Inner Join 0 to 1
                                                      condition expressions:
                                                        0 {_col1} {_col10} {_col11} {_col18} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col250} {_col259} {_col260} {_col263} {_col266} {_col275} {_col276} {_col279} {_col282}
                                                        1 
                                                      keys:
                                                        0 _col242 (type: int)
                                                        1 ib_income_band_sk (type: int)
                                                      outputColumnNames: _col1, _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col250, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282
                                                      input vertices:
                                                        1 Map 5
                                                      Statistics: Num rows: 52764356 Data size: 3944952576 Basic stats: COMPLETE Column stats: NONE
                                                      Map Join Operator
                                                        condition map:
                                                             Inner Join 0 to 1
                                                        condition expressions:
                                                          0 {_col1} {_col10} {_col11} {_col18} {_col107} {_col138} {_col168} {_col188} {_col197} {_col209} {_col259} {_col260} {_col263} {_col266} {_col275} {_col276} {_col279} {_col282}
                                                          1 
                                                        keys:
                                                          0 _col250 (type: int)
                                                          1 ib_income_band_sk (type: int)
                                                        outputColumnNames: _col1, _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282
                                                        input vertices:
                                                          1 Map 32
                                                        Statistics: Num rows: 58040792 Data size: 4339447808 Basic stats: COMPLETE Column stats: NONE
                                                        Reduce Output Operator
                                                          key expressions: _col1 (type: int)
                                                          sort order: +
                                                          Map-reduce partition columns: _col1 (type: int)
                                                          Statistics: Num rows: 58040792 Data size: 4339447808 Basic stats: COMPLETE Column stats: NONE
                                                          value expressions: _col10 (type: float), _col11 (type: float), _col18 (type: float), _col107 (type: int), _col138 (type: int), _col168 (type: string), _col188 (type: string), _col197 (type: string), _col209 (type: string), _col259 (type: string), _col260 (type: string), _col263 (type: string), _col266 (type: string), _col275 (type: string), _col276 (type: string), _col279 (type: string), _col282 (type: string)
                  Execution mode: vectorized
              Map 29 
                  Map Operator Tree:
                      TableScan
                        alias: store
                        filterExpr: ((s_store_sk is not null and s_store_name is not null) and s_zip is not null) (type: boolean)
                        Statistics: Num rows: 212 Data size: 405680 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ((s_store_sk is not null and s_store_name is not null) and s_zip is not null) (type: boolean)
                          Statistics: Num rows: 27 Data size: 51666 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: s_store_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: s_store_sk (type: int)
                            Statistics: Num rows: 27 Data size: 51666 Basic stats: COMPLETE Column stats: NONE
                            value expressions: s_store_name (type: string), s_zip (type: string)
                  Execution mode: vectorized
              Map 3 
                  Map Operator Tree:
                      TableScan
                        alias: promotion
                        filterExpr: p_promo_sk is not null (type: boolean)
                        Statistics: Num rows: 450 Data size: 530848 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: p_promo_sk is not null (type: boolean)
                          Statistics: Num rows: 225 Data size: 265424 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: p_promo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: p_promo_sk (type: int)
                            Statistics: Num rows: 225 Data size: 265424 Basic stats: COMPLETE Column stats: NONE
                  Execution mode: vectorized
              Map 30 
                  Map Operator Tree:
                      TableScan
                        alias: d1
                        filterExpr: (d_date_sk is not null and (d_year = 2000)) (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (d_date_sk is not null and (d_year = 2000)) (type: boolean)
                          Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: d_date_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: d_date_sk (type: int)
                            Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                          Select Operator
                            expressions: d_date_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                            Group By Operator
                              keys: _col0 (type: int)
                              mode: hash
                              outputColumnNames: _col0
                              Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                              Dynamic Partitioning Event Operator
                                Target Input: store_sales
                                Partition key expr: ss_sold_date_sk
                                Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                                Target column: ss_sold_date_sk
                                Target Vertex: Map 11
              Map 31 
                  Map Operator Tree:
                      TableScan
                        alias: d2
                        filterExpr: d_date_sk is not null (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: d_date_sk is not null (type: boolean)
                          Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: d_date_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: d_date_sk (type: int)
                            Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                            value expressions: d_year (type: int)
                  Execution mode: vectorized
              Map 32 
                  Map Operator Tree:
                      TableScan
                        alias: ib2
                        filterExpr: ib_income_band_sk is not null (type: boolean)
                        Statistics: Num rows: 20 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ib_income_band_sk is not null (type: boolean)
                          Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ib_income_band_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ib_income_band_sk (type: int)
                            Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                  Execution mode: vectorized
              Map 33 
                  Map Operator Tree:
                      TableScan
                        alias: catalog_returns
                        filterExpr: (cr_item_sk is not null and cr_order_number is not null) (type: boolean)
                        Statistics: Num rows: 28798881 Data size: 2942039156 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (cr_item_sk is not null and cr_order_number is not null) (type: boolean)
                          Statistics: Num rows: 7199721 Data size: 735509865 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: cr_item_sk (type: int), cr_order_number (type: int)
                            sort order: ++
                            Map-reduce partition columns: cr_item_sk (type: int), cr_order_number (type: int)
                            Statistics: Num rows: 7199721 Data size: 735509865 Basic stats: COMPLETE Column stats: NONE
                            value expressions: cr_refunded_cash (type: float), cr_reversed_charge (type: float), cr_store_credit (type: float)
                  Execution mode: vectorized
              Map 34 
                  Map Operator Tree:
                      TableScan
                        alias: item
                        filterExpr: (((i_item_sk is not null and (i_color) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate')) and i_current_price BETWEEN 35 AND 45) and i_current_price BETWEEN 36 AND 50) (type: boolean)
                        Statistics: Num rows: 48000 Data size: 68732712 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (((i_item_sk is not null and (i_color) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate')) and i_current_price BETWEEN 35 AND 45) and i_current_price BETWEEN 36 AND 50) (type: boolean)
                          Statistics: Num rows: 3000 Data size: 4295794 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: i_item_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: i_item_sk (type: int)
                            Statistics: Num rows: 3000 Data size: 4295794 Basic stats: COMPLETE Column stats: NONE
                            value expressions: i_current_price (type: float), i_color (type: string), i_product_name (type: string)
                  Execution mode: vectorized
              Map 35 
                  Map Operator Tree:
                      TableScan
                        alias: cd1
                        filterExpr: cd_demo_sk is not null (type: boolean)
                        Statistics: Num rows: 1920800 Data size: 718379200 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: cd_demo_sk is not null (type: boolean)
                          Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: cd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: cd_demo_sk (type: int)
                            Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: cd_marital_status (type: string)
                  Execution mode: vectorized
              Map 36 
                  Map Operator Tree:
                      TableScan
                        alias: ib1
                        filterExpr: ib_income_band_sk is not null (type: boolean)
                        Statistics: Num rows: 20 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ib_income_band_sk is not null (type: boolean)
                          Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ib_income_band_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ib_income_band_sk (type: int)
                            Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                  Execution mode: vectorized
              Map 37 
                  Map Operator Tree:
                      TableScan
                        alias: ib2
                        filterExpr: ib_income_band_sk is not null (type: boolean)
                        Statistics: Num rows: 20 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ib_income_band_sk is not null (type: boolean)
                          Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ib_income_band_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ib_income_band_sk (type: int)
                            Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                  Execution mode: vectorized
              Map 38 
                  Map Operator Tree:
                      TableScan
                        alias: cd2
                        filterExpr: cd_demo_sk is not null (type: boolean)
                        Statistics: Num rows: 1920800 Data size: 718379200 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: cd_demo_sk is not null (type: boolean)
                          Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: cd_demo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: cd_demo_sk (type: int)
                            Statistics: Num rows: 960400 Data size: 359189600 Basic stats: COMPLETE Column stats: NONE
                            value expressions: cd_marital_status (type: string)
                  Execution mode: vectorized
              Map 39 
                  Map Operator Tree:
                      TableScan
                        alias: ad1
                        filterExpr: ca_address_sk is not null (type: boolean)
                        Statistics: Num rows: 800000 Data size: 811903688 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ca_address_sk is not null (type: boolean)
                          Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ca_address_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ca_address_sk (type: int)
                            Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                            value expressions: ca_street_number (type: string), ca_street_name (type: string), ca_city (type: string), ca_zip (type: string)
                  Execution mode: vectorized
              Map 4 
                  Map Operator Tree:
                      TableScan
                        alias: store_sales
                        filterExpr: (((((((ss_item_sk is not null and ss_ticket_number is not null) and ss_customer_sk is not null) and ss_store_sk is not null) and ss_cdemo_sk is not null) and ss_promo_sk is not null) and ss_hdemo_sk is not null) and ss_addr_sk is not null) (type: boolean)
                        Statistics: Num rows: 550076554 Data size: 47370018896 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (((((((ss_item_sk is not null and ss_ticket_number is not null) and ss_customer_sk is not null) and ss_store_sk is not null) and ss_cdemo_sk is not null) and ss_promo_sk is not null) and ss_hdemo_sk is not null) and ss_addr_sk is not null) (type: boolean)
                          Statistics: Num rows: 2148737 Data size: 185039176 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ss_item_sk (type: int), ss_ticket_number (type: int)
                            sort order: ++
                            Map-reduce partition columns: ss_item_sk (type: int), ss_ticket_number (type: int)
                            Statistics: Num rows: 2148737 Data size: 185039176 Basic stats: COMPLETE Column stats: NONE
                            value expressions: 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_wholesale_cost (type: float), ss_list_price (type: float), ss_coupon_amt (type: float), ss_sold_date_sk (type: int)
                  Execution mode: vectorized
              Map 40 
                  Map Operator Tree:
                      TableScan
                        alias: promotion
                        filterExpr: p_promo_sk is not null (type: boolean)
                        Statistics: Num rows: 450 Data size: 530848 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: p_promo_sk is not null (type: boolean)
                          Statistics: Num rows: 225 Data size: 265424 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: p_promo_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: p_promo_sk (type: int)
                            Statistics: Num rows: 225 Data size: 265424 Basic stats: COMPLETE Column stats: NONE
                  Execution mode: vectorized
              Map 41 
                  Map Operator Tree:
                      TableScan
                        alias: d2
                        filterExpr: d_date_sk is not null (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: d_date_sk is not null (type: boolean)
                          Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: d_date_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: d_date_sk (type: int)
                            Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                            value expressions: d_year (type: int)
                  Execution mode: vectorized
              Map 42 
                  Map Operator Tree:
                      TableScan
                        alias: d3
                        filterExpr: d_date_sk is not null (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: d_date_sk is not null (type: boolean)
                          Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: d_date_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: d_date_sk (type: int)
                            Statistics: Num rows: 36525 Data size: 40871475 Basic stats: COMPLETE Column stats: NONE
                            value expressions: d_year (type: int)
                  Execution mode: vectorized
              Map 43 
                  Map Operator Tree:
                      TableScan
                        alias: ad2
                        filterExpr: ca_address_sk is not null (type: boolean)
                        Statistics: Num rows: 800000 Data size: 811903688 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ca_address_sk is not null (type: boolean)
                          Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ca_address_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ca_address_sk (type: int)
                            Statistics: Num rows: 400000 Data size: 405951844 Basic stats: COMPLETE Column stats: NONE
                            value expressions: ca_street_number (type: string), ca_street_name (type: string), ca_city (type: string), ca_zip (type: string)
                  Execution mode: vectorized
              Map 44 
                  Map Operator Tree:
                      TableScan
                        alias: d1
                        filterExpr: (d_date_sk is not null and (d_year = 2001)) (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (d_date_sk is not null and (d_year = 2001)) (type: boolean)
                          Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: d_date_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: d_date_sk (type: int)
                            Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                          Select Operator
                            expressions: d_date_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                            Group By Operator
                              keys: _col0 (type: int)
                              mode: hash
                              outputColumnNames: _col0
                              Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                              Dynamic Partitioning Event Operator
                                Target Input: store_sales
                                Partition key expr: ss_sold_date_sk
                                Statistics: Num rows: 18262 Data size: 20435178 Basic stats: COMPLETE Column stats: NONE
                                Target column: ss_sold_date_sk
                                Target Vertex: Map 4
              Map 45 
                  Map Operator Tree:
                      TableScan
                        alias: store
                        filterExpr: ((s_store_sk is not null and s_store_name is not null) and s_zip is not null) (type: boolean)
                        Statistics: Num rows: 212 Data size: 405680 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ((s_store_sk is not null and s_store_name is not null) and s_zip is not null) (type: boolean)
                          Statistics: Num rows: 27 Data size: 51666 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: s_store_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: s_store_sk (type: int)
                            Statistics: Num rows: 27 Data size: 51666 Basic stats: COMPLETE Column stats: NONE
                            value expressions: s_store_name (type: string), s_zip (type: string)
                  Execution mode: vectorized
              Map 5 
                  Map Operator Tree:
                      TableScan
                        alias: ib1
                        filterExpr: ib_income_band_sk is not null (type: boolean)
                        Statistics: Num rows: 20 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: ib_income_band_sk is not null (type: boolean)
                          Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: ib_income_band_sk (type: int)
                            sort order: +
                            Map-reduce partition columns: ib_income_band_sk (type: int)
                            Statistics: Num rows: 10 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                  Execution mode: vectorized
              Map 6 
                  Map Operator Tree:
                      TableScan
                        alias: catalog_sales
                        filterExpr: (cs_item_sk is not null and cs_order_number is not null) (type: boolean)
                        Statistics: Num rows: 286549727 Data size: 37743959324 Basic stats: COMPLETE Column stats: NONE
                        Filter Operator
                          predicate: (cs_item_sk is not null and cs_order_number is not null) (type: boolean)
                          Statistics: Num rows: 71637432 Data size: 9435989863 Basic stats: COMPLETE Column stats: NONE
                          Map Join Operator
                            condition map:
                                 Inner Join 0 to 1
                            condition expressions:
                              0 {cs_item_sk} {cs_ext_list_price}
                              1 {cr_refunded_cash} {cr_reversed_charge} {cr_store_credit}
                            keys:
                              0 cs_item_sk (type: int), cs_order_number (type: int)
                              1 cr_item_sk (type: int), cr_order_number (type: int)
                            outputColumnNames: _col14, _col24, _col59, _col60, _col61
                            input vertices:
                              1 Map 21
                            Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                            Select Operator
                              expressions: _col14 (type: int), _col24 (type: float), _col59 (type: float), _col60 (type: float), _col61 (type: float)
                              outputColumnNames: _col14, _col24, _col59, _col60, _col61
                              Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                              Group By Operator
                                aggregations: sum(_col24), sum(((_col59 + _col60) + _col61))
                                keys: _col14 (type: int)
                                mode: hash
                                outputColumnNames: _col0, _col1, _col2
                                Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                                Reduce Output Operator
                                  key expressions: _col0 (type: int)
                                  sort order: +
                                  Map-reduce partition columns: _col0 (type: int)
                                  Statistics: Num rows: 78801176 Data size: 10379589632 Basic stats: COMPLETE Column stats: NONE
                                  value expressions: _col1 (type: double), _col2 (type: double)
                  Execution mode: vectorized
              Reducer 10 
                  Reduce Operator Tree:
                    Select Operator
                      expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: string), VALUE._col8 (type: string), 2000 (type: int), VALUE._col10 (type: bigint), VALUE._col11 (type: double), VALUE._col12 (type: double), VALUE._col13 (type: double), VALUE._col14 (type: double), VALUE._col15 (type: double), VALUE._col16 (type: double), 2001 (type: int), KEY.reducesinkkey2 (type: bigint)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                      Statistics: Num rows: 182888 Data size: 13673711 Basic stats: COMPLETE Column stats: NONE
                      File Output Operator
                        compressed: false
                        Statistics: Num rows: 182888 Data size: 13673711 Basic stats: COMPLETE Column stats: NONE
                        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
              Reducer 19 
                  Reduce Operator Tree:
                    Join Operator
                      condition map:
                           Inner Join 0 to 1
                           Inner Join 0 to 2
                      condition expressions:
                        0 {VALUE._col9} {VALUE._col10} {VALUE._col17} {VALUE._col106} {VALUE._col137} {VALUE._col167} {VALUE._col187} {VALUE._col196} {VALUE._col208} {VALUE._col258} {VALUE._col259} {VALUE._col262} {VALUE._col265} {VALUE._col274} {VALUE._col275} {VALUE._col278} {VALUE._col281}
                        1 {KEY.reducesinkkey0} {VALUE._col4} {VALUE._col16} {VALUE._col20}
                        2 
                      outputColumnNames: _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282, _col301, _col306, _col318, _col322
                      Statistics: Num rows: 127689744 Data size: 9546785792 Basic stats: COMPLETE Column stats: NONE
                      Filter Operator
                        predicate: ((((_col197 <> _col209) and (_col318) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate')) and _col306 BETWEEN 35 AND 45) and _col306 BETWEEN 36 AND 50) (type: boolean)
                        Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                        Select Operator
                          expressions: _col322 (type: string), _col301 (type: int), _col168 (type: string), _col188 (type: string), _col259 (type: string), _col260 (type: string), _col263 (type: string), _col266 (type: string), _col275 (type: string), _col276 (type: string), _col279 (type: string), _col282 (type: string), 2001 (type: int), _col107 (type: int), _col138 (type: int), _col10 (type: float), _col11 (type: float), _col18 (type: float)
                          outputColumnNames: _col322, _col301, _col168, _col188, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282, _col76, _col107, _col138, _col10, _col11, _col18
                          Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                          Group By Operator
                            aggregations: count(), sum(_col10), sum(_col11), sum(_col18)
                            keys: _col322 (type: string), _col301 (type: int), _col168 (type: string), _col188 (type: string), _col259 (type: string), _col260 (type: string), _col263 (type: string), _col266 (type: string), _col275 (type: string), _col276 (type: string), _col279 (type: string), _col282 (type: string), _col76 (type: int), _col107 (type: int), _col138 (type: int)
                            mode: hash
                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
                            Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                            Reduce Output Operator
                              key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: int), _col13 (type: int), _col14 (type: int)
                              sort order: +++++++++++++++
                              Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: int), _col13 (type: int), _col14 (type: int)
                              Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                              value expressions: _col15 (type: bigint), _col16 (type: double), _col17 (type: double), _col18 (type: double)
              Reducer 20 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: count(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2), sum(VALUE._col3)
                      keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: string), KEY._col5 (type: string), KEY._col6 (type: string), KEY._col7 (type: string), KEY._col8 (type: string), KEY._col9 (type: string), KEY._col10 (type: string), KEY._col11 (type: string), KEY._col12 (type: int), KEY._col13 (type: int), KEY._col14 (type: int)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
                      Statistics: Num rows: 7980609 Data size: 596674112 Basic stats: COMPLETE Column stats: NONE
                      Select Operator
                        expressions: _col1 (type: int), _col12 (type: int), _col15 (type: bigint), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col2 (type: string), _col3 (type: string)
                        outputColumnNames: _col1, _col12, _col15, _col16, _col17, _col18, _col2, _col3
                        Statistics: Num rows: 7980609 Data size: 596674112 Basic stats: COMPLETE Column stats: NONE
                        Reduce Output Operator
                          key expressions: _col1 (type: int), _col2 (type: string), _col3 (type: string)
                          sort order: +++
                          Map-reduce partition columns: _col1 (type: int), _col2 (type: string), _col3 (type: string)
                          Statistics: Num rows: 7980609 Data size: 596674112 Basic stats: COMPLETE Column stats: NONE
                          value expressions: _col12 (type: int), _col15 (type: bigint), _col16 (type: double), _col17 (type: double), _col18 (type: double)
              Reducer 23 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: sum(VALUE._col0), sum(VALUE._col1)
                      keys: KEY._col0 (type: int)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col2
                      Statistics: Num rows: 39400588 Data size: 5189794816 Basic stats: COMPLETE Column stats: NONE
                      Filter Operator
                        predicate: (_col1 > (2 * _col2)) (type: boolean)
                        Statistics: Num rows: 13133529 Data size: 1729931561 Basic stats: COMPLETE Column stats: NONE
                        Select Operator
                          expressions: _col0 (type: int)
                          outputColumnNames: _col0
                          Statistics: Num rows: 13133529 Data size: 1729931561 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: _col0 (type: int)
                            sort order: +
                            Map-reduce partition columns: _col0 (type: int)
                            Statistics: Num rows: 13133529 Data size: 1729931561 Basic stats: COMPLETE Column stats: NONE
              Reducer 7 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: sum(VALUE._col0), sum(VALUE._col1)
                      keys: KEY._col0 (type: int)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col2
                      Statistics: Num rows: 39400588 Data size: 5189794816 Basic stats: COMPLETE Column stats: NONE
                      Filter Operator
                        predicate: (_col1 > (2 * _col2)) (type: boolean)
                        Statistics: Num rows: 13133529 Data size: 1729931561 Basic stats: COMPLETE Column stats: NONE
                        Select Operator
                          expressions: _col0 (type: int)
                          outputColumnNames: _col0
                          Statistics: Num rows: 13133529 Data size: 1729931561 Basic stats: COMPLETE Column stats: NONE
                          Reduce Output Operator
                            key expressions: _col0 (type: int)
                            sort order: +
                            Map-reduce partition columns: _col0 (type: int)
                            Statistics: Num rows: 13133529 Data size: 1729931561 Basic stats: COMPLETE Column stats: NONE
              Reducer 8 
                  Reduce Operator Tree:
                    Join Operator
                      condition map:
                           Inner Join 0 to 1
                           Inner Join 0 to 2
                      condition expressions:
                        0 {VALUE._col9} {VALUE._col10} {VALUE._col17} {VALUE._col106} {VALUE._col137} {VALUE._col167} {VALUE._col187} {VALUE._col196} {VALUE._col208} {VALUE._col258} {VALUE._col259} {VALUE._col262} {VALUE._col265} {VALUE._col274} {VALUE._col275} {VALUE._col278} {VALUE._col281}
                        1 {KEY.reducesinkkey0} {VALUE._col4} {VALUE._col16} {VALUE._col20}
                        2 
                      outputColumnNames: _col10, _col11, _col18, _col107, _col138, _col168, _col188, _col197, _col209, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282, _col301, _col306, _col318, _col322
                      Statistics: Num rows: 127689744 Data size: 9546785792 Basic stats: COMPLETE Column stats: NONE
                      Filter Operator
                        predicate: ((((_col197 <> _col209) and (_col318) IN ('maroon', 'burnished', 'dim', 'steel', 'navajo', 'chocolate')) and _col306 BETWEEN 35 AND 45) and _col306 BETWEEN 36 AND 50) (type: boolean)
                        Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                        Select Operator
                          expressions: _col322 (type: string), _col301 (type: int), _col168 (type: string), _col188 (type: string), _col259 (type: string), _col260 (type: string), _col263 (type: string), _col266 (type: string), _col275 (type: string), _col276 (type: string), _col279 (type: string), _col282 (type: string), 2000 (type: int), _col107 (type: int), _col138 (type: int), _col10 (type: float), _col11 (type: float), _col18 (type: float)
                          outputColumnNames: _col322, _col301, _col168, _col188, _col259, _col260, _col263, _col266, _col275, _col276, _col279, _col282, _col76, _col107, _col138, _col10, _col11, _col18
                          Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                          Group By Operator
                            aggregations: count(), sum(_col10), sum(_col11), sum(_col18)
                            keys: _col322 (type: string), _col301 (type: int), _col168 (type: string), _col188 (type: string), _col259 (type: string), _col260 (type: string), _col263 (type: string), _col266 (type: string), _col275 (type: string), _col276 (type: string), _col279 (type: string), _col282 (type: string), _col76 (type: int), _col107 (type: int), _col138 (type: int)
                            mode: hash
                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
                            Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                            Reduce Output Operator
                              key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: int), _col13 (type: int), _col14 (type: int)
                              sort order: +++++++++++++++
                              Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: int), _col13 (type: int), _col14 (type: int)
                              Statistics: Num rows: 15961218 Data size: 1193348224 Basic stats: COMPLETE Column stats: NONE
                              value expressions: _col15 (type: bigint), _col16 (type: double), _col17 (type: double), _col18 (type: double)
              Reducer 9 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: count(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2), sum(VALUE._col3)
                      keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: string), KEY._col3 (type: string), KEY._col4 (type: string), KEY._col5 (type: string), KEY._col6 (type: string), KEY._col7 (type: string), KEY._col8 (type: string), KEY._col9 (type: string), KEY._col10 (type: string), KEY._col11 (type: string), KEY._col12 (type: int), KEY._col13 (type: int), KEY._col14 (type: int)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
                      Statistics: Num rows: 7980609 Data size: 596674112 Basic stats: COMPLETE Column stats: NONE
                      Select Operator
                        expressions: _col0 (type: string), _col1 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: int), _col15 (type: bigint), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: string)
                        outputColumnNames: _col0, _col1, _col10, _col11, _col12, _col15, _col16, _col17, _col18, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                        Statistics: Num rows: 7980609 Data size: 596674112 Basic stats: COMPLETE Column stats: NONE
                        Map Join Operator
                          condition map:
                               Inner Join 0 to 1
                          condition expressions:
                            0 {_col0} {_col2} {_col3} {_col4} {_col5} {_col6} {_col7} {_col8} {_col9} {_col10} {_col11} {_col12} {_col15} {_col16} {_col17} {_col18}
                            1 {_col2} {_col3} {_col12} {_col15} {_col16} {_col17} {_col18}
                          keys:
                            0 _col1 (type: int), _col2 (type: string), _col3 (type: string)
                            1 _col1 (type: int), _col2 (type: string), _col3 (type: string)
                          outputColumnNames: _col0, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col15, _col16, _col17, _col18, _col21, _col22, _col31, _col34, _col35, _col36, _col37
                          input vertices:
                            1 Reducer 20
                          Statistics: Num rows: 8778670 Data size: 656341568 Basic stats: COMPLETE Column stats: NONE
                          Filter Operator
                            predicate: (((((_col12 = 2000) and (_col31 = 2001)) and (_col34 <= _col15)) and (_col2 = _col21)) and (_col3 = _col22)) (type: boolean)
                            Statistics: Num rows: 182888 Data size: 13673711 Basic stats: COMPLETE Column stats: NONE
                            Select Operator
                              expressions: _col0 (type: string), _col2 (type: string), _col11 (type: string), _col15 (type: bigint), _col16 (type: double), _col17 (type: double), _col18 (type: double), _col35 (type: double), _col36 (type: double), _col37 (type: double), _col3 (type: string), _col34 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: string), _col10 (type: string)
                              outputColumnNames: _col0, _col1, _col10, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col2, _col20, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                              Statistics: Num rows: 182888 Data size: 13673711 Basic stats: COMPLETE Column stats: NONE
                              Reduce Output Operator
                                key expressions: _col0 (type: string), _col1 (type: string), _col20 (type: bigint)
                                sort order: +++
                                Statistics: Num rows: 182888 Data size: 13673711 Basic stats: COMPLETE Column stats: NONE
                                value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: string), _col10 (type: string), _col12 (type: bigint), _col13 (type: double), _col14 (type: double), _col15 (type: double), _col16 (type: double), _col17 (type: double), _col18 (type: double)
      
        Stage: Stage-0
          Fetch Operator
            limit: -1
            Processor Tree:
              ListSink
      

      Query

      explain  
      select cs1.product_name ,cs1.store_name ,cs1.store_zip ,cs1.b_street_number ,cs1.b_streen_name ,cs1.b_city
           ,cs1.b_zip ,cs1.c_street_number ,cs1.c_street_name ,cs1.c_city ,cs1.c_zip ,cs1.syear ,cs1.cnt
           ,cs1.s1 ,cs1.s2 ,cs1.s3
           ,cs2.s1 ,cs2.s2 ,cs2.s3 ,cs2.syear ,cs2.cnt
      from
      (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_name
           ,s_zip as store_zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name
           ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number
           ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip
           ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt
           ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3
        FROM   store_sales
              JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number
              JOIN customer ON store_sales.ss_customer_sk = customer.c_customer_sk
              JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk
              JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk 
              JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk
              JOIN store ON store_sales.ss_store_sk = store.s_store_sk
              JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk
              JOIN customer_demographics cd2 ON customer.c_current_cdemo_sk = cd2.cd_demo_sk
              JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk
              JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk
              JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk
              JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk
              JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk
              JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk
              JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk
              JOIN item ON store_sales.ss_item_sk = item.i_item_sk
              JOIN
       (select cs_item_sk
              ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund
        from catalog_sales JOIN catalog_returns
        ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk
          and catalog_sales.cs_order_number = catalog_returns.cr_order_number
        group by cs_item_sk
        having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)) cs_ui
      ON store_sales.ss_item_sk = cs_ui.cs_item_sk
        WHERE  
               cd1.cd_marital_status <> cd2.cd_marital_status and
               i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and
               i_current_price between 35 and 35 + 10 and
               i_current_price between 35 + 1 and 35 + 15
      group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number
             ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number
             ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year
      ) cs1
      JOIN
      (select i_product_name as product_name ,i_item_sk as item_sk ,s_store_name as store_name
           ,s_zip as store_zip ,ad1.ca_street_number as b_street_number ,ad1.ca_street_name as b_streen_name
           ,ad1.ca_city as b_city ,ad1.ca_zip as b_zip ,ad2.ca_street_number as c_street_number
           ,ad2.ca_street_name as c_street_name ,ad2.ca_city as c_city ,ad2.ca_zip as c_zip
           ,d1.d_year as syear ,d2.d_year as fsyear ,d3.d_year as s2year ,count(*) as cnt
           ,sum(ss_wholesale_cost) as s1 ,sum(ss_list_price) as s2 ,sum(ss_coupon_amt) as s3
        FROM   store_sales
              JOIN store_returns ON store_sales.ss_item_sk = store_returns.sr_item_sk and store_sales.ss_ticket_number = store_returns.sr_ticket_number
              JOIN customer ON store_sales.ss_customer_sk = customer.c_customer_sk
              JOIN date_dim d1 ON store_sales.ss_sold_date_sk = d1.d_date_sk
              JOIN date_dim d2 ON customer.c_first_sales_date_sk = d2.d_date_sk 
              JOIN date_dim d3 ON customer.c_first_shipto_date_sk = d3.d_date_sk
              JOIN store ON store_sales.ss_store_sk = store.s_store_sk
              JOIN customer_demographics cd1 ON store_sales.ss_cdemo_sk= cd1.cd_demo_sk
              JOIN customer_demographics cd2 ON customer.c_current_cdemo_sk = cd2.cd_demo_sk
              JOIN promotion ON store_sales.ss_promo_sk = promotion.p_promo_sk
              JOIN household_demographics hd1 ON store_sales.ss_hdemo_sk = hd1.hd_demo_sk
              JOIN household_demographics hd2 ON customer.c_current_hdemo_sk = hd2.hd_demo_sk
              JOIN customer_address ad1 ON store_sales.ss_addr_sk = ad1.ca_address_sk
              JOIN customer_address ad2 ON customer.c_current_addr_sk = ad2.ca_address_sk
              JOIN income_band ib1 ON hd1.hd_income_band_sk = ib1.ib_income_band_sk
              JOIN income_band ib2 ON hd2.hd_income_band_sk = ib2.ib_income_band_sk
              JOIN item ON store_sales.ss_item_sk = item.i_item_sk
              JOIN
       (select cs_item_sk
              ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund
        from catalog_sales JOIN catalog_returns
        ON catalog_sales.cs_item_sk = catalog_returns.cr_item_sk
          and catalog_sales.cs_order_number = catalog_returns.cr_order_number
        group by cs_item_sk
        having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)) cs_ui
      ON store_sales.ss_item_sk = cs_ui.cs_item_sk
        WHERE  
               cd1.cd_marital_status <> cd2.cd_marital_status and
               i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and
               i_current_price between 35 and 35 + 10 and
               i_current_price between 35 + 1 and 35 + 15
      group by i_product_name ,i_item_sk ,s_store_name ,s_zip ,ad1.ca_street_number
             ,ad1.ca_street_name ,ad1.ca_city ,ad1.ca_zip ,ad2.ca_street_number
             ,ad2.ca_street_name ,ad2.ca_city ,ad2.ca_zip ,d1.d_year ,d2.d_year ,d3.d_year
      ) cs2
      ON cs1.item_sk=cs2.item_sk
      where 
           cs1.syear = 2000 and
           cs2.syear = 2000 + 1 and
           cs2.cnt <= cs1.cnt and
           cs1.store_name = cs2.store_name and
           cs1.store_zip = cs2.store_zip
      order by cs1.product_name ,cs1.store_name ,cs2.cnt
      

      Attachments

        1. query64_oom_trim.txt
          968 kB
          Mostafa Mokhtar

        Activity

          People

            prasanth_j Prasanth Jayachandran
            mmokhtar Mostafa Mokhtar
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: