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

[jira] [Created] (HIVE-9624) NullPointerException in MapJoinOperator.processOp(MapJoinOperator.java:253) for TPC-DS Q75 against un-partitioned schema

Mostafa Mokhtar created HIVE-9624:
-------------------------------------

             Summary: NullPointerException in MapJoinOperator.processOp(MapJoinOperator.java:253) for TPC-DS Q75 against un-partitioned schema
                 Key: HIVE-9624
                 URL: https://issues.apache.org/jira/browse/HIVE-9624
             Project: Hive
          Issue Type: Bug
          Components: Query Processor
    Affects Versions: 0.14.0
            Reporter: Mostafa Mokhtar
            Assignee: Gunther Hagleitner
             Fix For: 1.2.0


Running TPC-DS Q75 against a non-partitioned schema fails with 
{code}
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Unexpected exception: null
	at org.apache.hadoop.hive.ql.exec.MapJoinOperator.processOp(MapJoinOperator.java:314)
	at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:815)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.internalForward(CommonJoinOperator.java:638)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.createForwardJoinObject(CommonJoinOperator.java:433)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genObject(CommonJoinOperator.java:525)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genObject(CommonJoinOperator.java:522)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.genJoinObject(CommonJoinOperator.java:451)
	at org.apache.hadoop.hive.ql.exec.CommonJoinOperator.checkAndGenObject(CommonJoinOperator.java:752)
	at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.joinObject(CommonMergeJoinOperator.java:248)
	at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.joinOneGroup(CommonMergeJoinOperator.java:213)
	at org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator.processOp(CommonMergeJoinOperator.java:196)
	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource$GroupIterator.next(ReduceRecordSource.java:328)
	... 16 more
Caused by: java.lang.NullPointerException
	at org.apache.hadoop.hive.ql.exec.MapJoinOperator.processOp(MapJoinOperator.java:253)
	... 27 more
{code}

This line maps to hashMapRowGetters = new ReusableGetAdaptor[mapJoinTables.length] in the code snippet below
{code}
     alias = (byte) tag;
      if (hashMapRowGetters == null) {
        hashMapRowGetters = new ReusableGetAdaptor[mapJoinTables.length];
        MapJoinKey refKey = getRefKey(alias);
        for (byte pos = 0; pos < order.length; pos++) {
          if (pos != alias) {
            hashMapRowGetters[pos] = mapJoinTables[pos].createGetter(refKey);
          }
        }
      }
{code}

Query 
{code}

WITH all_sales AS (
 SELECT d_year
       ,i_brand_id
       ,i_class_id
       ,i_category_id
       ,i_manufact_id
       ,SUM(sales_cnt) AS sales_cnt
       ,SUM(sales_amt) AS sales_amt
 FROM (SELECT d_year
             ,i_brand_id
             ,i_class_id
             ,i_category_id
             ,i_manufact_id
             ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt
             ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt
       FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk
                          JOIN date_dim ON d_date_sk=cs_sold_date_sk
                          LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number 
                                                    AND cs_item_sk=cr_item_sk)
       WHERE i_category='Sports'
       UNION ALL
       SELECT d_year
             ,i_brand_id
             ,i_class_id
             ,i_category_id
             ,i_manufact_id
             ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt
             ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt
       FROM store_sales JOIN item ON i_item_sk=ss_item_sk
                        JOIN date_dim ON d_date_sk=ss_sold_date_sk
                        LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number 
                                                AND ss_item_sk=sr_item_sk)
       WHERE i_category='Sports'
       UNION ALL
       SELECT d_year
             ,i_brand_id
             ,i_class_id
             ,i_category_id
             ,i_manufact_id
             ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt
             ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt
       FROM web_sales JOIN item ON i_item_sk=ws_item_sk
                      JOIN date_dim ON d_date_sk=ws_sold_date_sk
                      LEFT JOIN web_returns ON (ws_order_number=wr_order_number 
                                            AND ws_item_sk=wr_item_sk)
       WHERE i_category='Sports') sales_detail
 GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id)
 SELECT  prev_yr.d_year AS prev_year
                          ,curr_yr.d_year AS year
                          ,curr_yr.i_brand_id
                          ,curr_yr.i_class_id
                          ,curr_yr.i_category_id
                          ,curr_yr.i_manufact_id
                          ,prev_yr.sales_cnt AS prev_yr_cnt
                          ,curr_yr.sales_cnt AS curr_yr_cnt
                          ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff
                          ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff
 FROM all_sales curr_yr, all_sales prev_yr
 WHERE curr_yr.i_brand_id=prev_yr.i_brand_id
   AND curr_yr.i_class_id=prev_yr.i_class_id
   AND curr_yr.i_category_id=prev_yr.i_category_id
   AND curr_yr.i_manufact_id=prev_yr.i_manufact_id
   AND curr_yr.d_year=2002
   AND prev_yr.d_year=2002-1
   AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9
 ORDER BY sales_cnt_diff
 limit 100
{code}

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

STAGE PLANS:
  Stage: Stage-1
    Tez
      Edges:
        Map 1 <- Map 6 (BROADCAST_EDGE)
        Map 14 <- Map 16 (BROADCAST_EDGE)
        Map 18 <- Reducer 15 (BROADCAST_EDGE), Union 3 (CONTAINS)
        Map 19 <- Map 23 (BROADCAST_EDGE)
        Map 26 <- Map 28 (BROADCAST_EDGE)
        Map 31 <- Map 33 (BROADCAST_EDGE)
        Map 35 <- Reducer 32 (BROADCAST_EDGE), Union 21 (CONTAINS)
        Map 9 <- Map 11 (BROADCAST_EDGE)
        Reducer 10 <- Map 12 (SIMPLE_EDGE), Map 13 (BROADCAST_EDGE), Map 9 (SIMPLE_EDGE), Union 3 (CONTAINS)
        Reducer 15 <- Map 14 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE), Map 8 (BROADCAST_EDGE), Union 3 (CONTAINS)
        Reducer 20 <- Map 19 (SIMPLE_EDGE), Map 24 (SIMPLE_EDGE), Map 25 (BROADCAST_EDGE), Union 21 (CONTAINS)
        Reducer 22 <- Union 21 (SIMPLE_EDGE)
        Reducer 27 <- Map 26 (SIMPLE_EDGE), Map 29 (SIMPLE_EDGE), Map 30 (BROADCAST_EDGE), Union 21 (CONTAINS)
        Reducer 32 <- Map 31 (SIMPLE_EDGE), Map 34 (SIMPLE_EDGE)
        Reducer 4 <- Reducer 22 (BROADCAST_EDGE), Union 3 (SIMPLE_EDGE)
        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
      DagName: mmokhtar_20150207174141_8f167b31-c893-4c6e-86d6-855d20744d92:1
      Vertices:
        Map 1 
            Map Operator Tree:
                TableScan
                  alias: catalog_sales
                  filterExpr: (cs_sold_date_sk is not null and cs_item_sk is not null) (type: boolean)
                  Statistics: Num rows: 817736652 Data size: 16354733056 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (cs_sold_date_sk is not null and cs_item_sk is not null) (type: boolean)
                    Statistics: Num rows: 817736652 Data size: 16348976724 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: cs_sold_date_sk (type: int), cs_item_sk (type: int), cs_order_number (type: int), cs_quantity (type: int), cs_ext_sales_price (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                      Statistics: Num rows: 817736652 Data size: 16348976724 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col0 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col1, _col2, _col3, _col4, _col6
                        input vertices:
                          1 Map 6
                        Statistics: Num rows: 148779579 Data size: 2380473264 Basic stats: COMPLETE Column stats: COMPLETE
                        Reduce Output Operator
                          key expressions: _col2 (type: int), _col1 (type: int)
                          sort order: ++
                          Map-reduce partition columns: _col2 (type: int), _col1 (type: int)
                          Statistics: Num rows: 148779579 Data size: 2380473264 Basic stats: COMPLETE Column stats: COMPLETE
                          value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int)
            Execution mode: vectorized
        Map 11 
            Map Operator Tree:
                TableScan
                  alias: date_dim
                  filterExpr: ((d_year = 2002) and d_date_sk is not null) (type: boolean)
                  Statistics: Num rows: 45363 Data size: 362905 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((d_year = 2002) and d_date_sk is not null) (type: boolean)
                    Statistics: Num rows: 405 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: d_date_sk (type: int)
                      outputColumnNames: _col0
                      Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: 2002 (type: int)
            Execution mode: vectorized
        Map 12 
            Map Operator Tree:
                TableScan
                  alias: store_returns
                  filterExpr: sr_item_sk is not null (type: boolean)
                  Statistics: Num rows: 167243952 Data size: 2675903232 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: sr_item_sk is not null (type: boolean)
                    Statistics: Num rows: 167243952 Data size: 2667828428 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: sr_item_sk (type: int), sr_ticket_number (type: int), sr_return_quantity (type: int), sr_return_amt (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Statistics: Num rows: 167243952 Data size: 2667828428 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col1 (type: int), _col0 (type: int)
                        sort order: ++
                        Map-reduce partition columns: _col1 (type: int), _col0 (type: int)
                        Statistics: Num rows: 167243952 Data size: 2667828428 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col2 (type: int), _col3 (type: float)
            Execution mode: vectorized
        Map 13 
            Map Operator Tree:
                TableScan
                  alias: item
                  filterExpr: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                  Statistics: Num rows: 118835 Data size: 3089722 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                    Statistics: Num rows: 11836 Data size: 1301772 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: i_item_sk (type: int), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int), i_manufact_id (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col5
                      Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col5 (type: int)
            Execution mode: vectorized
        Map 14 
            Map Operator Tree:
                TableScan
                  alias: web_sales
                  filterExpr: (ws_sold_date_sk is not null and ws_item_sk is not null) (type: boolean)
                  Statistics: Num rows: 447759411 Data size: 8955188224 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (ws_sold_date_sk is not null and ws_item_sk is not null) (type: boolean)
                    Statistics: Num rows: 447759411 Data size: 8955044072 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: ws_sold_date_sk (type: int), ws_item_sk (type: int), ws_order_number (type: int), ws_quantity (type: int), ws_ext_sales_price (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                      Statistics: Num rows: 447759411 Data size: 8955044072 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col0 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col1, _col2, _col3, _col4, _col6
                        input vertices:
                          1 Map 16
                        Statistics: Num rows: 81465661 Data size: 1303450576 Basic stats: COMPLETE Column stats: COMPLETE
                        Reduce Output Operator
                          key expressions: _col2 (type: int), _col1 (type: int)
                          sort order: ++
                          Map-reduce partition columns: _col2 (type: int), _col1 (type: int)
                          Statistics: Num rows: 81465661 Data size: 1303450576 Basic stats: COMPLETE Column stats: COMPLETE
                          value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int)
            Execution mode: vectorized
        Map 16 
            Map Operator Tree:
                TableScan
                  alias: date_dim
                  filterExpr: ((d_year = 2002) and d_date_sk is not null) (type: boolean)
                  Statistics: Num rows: 45363 Data size: 362905 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((d_year = 2002) and d_date_sk is not null) (type: boolean)
                    Statistics: Num rows: 405 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: d_date_sk (type: int)
                      outputColumnNames: _col0
                      Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: 2002 (type: int)
            Execution mode: vectorized
        Map 17 
            Map Operator Tree:
                TableScan
                  alias: web_returns
                  filterExpr: wr_item_sk is not null (type: boolean)
                  Statistics: Num rows: 50457044 Data size: 807312704 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: wr_item_sk is not null (type: boolean)
                    Statistics: Num rows: 50457044 Data size: 804725540 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: wr_item_sk (type: int), wr_order_number (type: int), wr_return_quantity (type: int), wr_return_amt (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Statistics: Num rows: 50457044 Data size: 804725540 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col1 (type: int), _col0 (type: int)
                        sort order: ++
                        Map-reduce partition columns: _col1 (type: int), _col0 (type: int)
                        Statistics: Num rows: 50457044 Data size: 804725540 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col2 (type: int), _col3 (type: float)
            Execution mode: vectorized
        Map 18 
            Map Operator Tree:
                TableScan
                  alias: item
                  filterExpr: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                  Filter Operator
                    predicate: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                    Select Operator
                      expressions: i_item_sk (type: int), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int), i_manufact_id (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col5
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col1 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col3, _col4, _col6, _col9, _col10, _col12, _col13, _col14, _col16
                        input vertices:
                          0 Reducer 15
                        Select Operator
                          expressions: _col6 (type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col16 (type: int), (_col3 - COALESCE(_col9,0)) (type: int), (UDFToDouble(_col4) - COALESCE(_col10,0.0)) (type: double)
                          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                          Group By Operator
                            aggregations: sum(_col5), sum(_col6)
                            keys: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                            mode: hash
                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                            Reduce Output Operator
                              key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                              sort order: +++++
                              Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                              value expressions: _col5 (type: bigint), _col6 (type: double)
            Execution mode: vectorized
        Map 19 
            Map Operator Tree:
                TableScan
                  alias: catalog_sales
                  filterExpr: (cs_sold_date_sk is not null and cs_item_sk is not null) (type: boolean)
                  Statistics: Num rows: 817736652 Data size: 16354733056 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (cs_sold_date_sk is not null and cs_item_sk is not null) (type: boolean)
                    Statistics: Num rows: 817736652 Data size: 16348976724 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: cs_sold_date_sk (type: int), cs_item_sk (type: int), cs_order_number (type: int), cs_quantity (type: int), cs_ext_sales_price (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                      Statistics: Num rows: 817736652 Data size: 16348976724 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col0 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col1, _col2, _col3, _col4, _col6
                        input vertices:
                          1 Map 23
                        Statistics: Num rows: 148779579 Data size: 2380473264 Basic stats: COMPLETE Column stats: COMPLETE
                        Reduce Output Operator
                          key expressions: _col2 (type: int), _col1 (type: int)
                          sort order: ++
                          Map-reduce partition columns: _col2 (type: int), _col1 (type: int)
                          Statistics: Num rows: 148779579 Data size: 2380473264 Basic stats: COMPLETE Column stats: COMPLETE
                          value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int)
            Execution mode: vectorized
        Map 23 
            Map Operator Tree:
                TableScan
                  alias: date_dim
                  filterExpr: ((d_year = 2001) and d_date_sk is not null) (type: boolean)
                  Statistics: Num rows: 45363 Data size: 362905 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((d_year = 2001) and d_date_sk is not null) (type: boolean)
                    Statistics: Num rows: 405 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: d_date_sk (type: int)
                      outputColumnNames: _col0
                      Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: 2001 (type: int)
            Execution mode: vectorized
        Map 24 
            Map Operator Tree:
                TableScan
                  alias: catalog_returns
                  filterExpr: cr_item_sk is not null (type: boolean)
                  Statistics: Num rows: 108409176 Data size: 1734546816 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: cr_item_sk is not null (type: boolean)
                    Statistics: Num rows: 108409176 Data size: 1729935996 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: cr_item_sk (type: int), cr_order_number (type: int), cr_return_quantity (type: int), cr_return_amount (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Statistics: Num rows: 108409176 Data size: 1729935996 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col1 (type: int), _col0 (type: int)
                        sort order: ++
                        Map-reduce partition columns: _col1 (type: int), _col0 (type: int)
                        Statistics: Num rows: 108409176 Data size: 1729935996 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col2 (type: int), _col3 (type: float)
            Execution mode: vectorized
        Map 25 
            Map Operator Tree:
                TableScan
                  alias: item
                  filterExpr: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                  Statistics: Num rows: 118835 Data size: 3089722 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                    Statistics: Num rows: 11836 Data size: 1301772 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: i_item_sk (type: int), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int), i_manufact_id (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col5
                      Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col5 (type: int)
            Execution mode: vectorized
        Map 26 
            Map Operator Tree:
                TableScan
                  alias: store_sales
                  filterExpr: (ss_sold_date_sk is not null and ss_item_sk is not null) (type: boolean)
                  Statistics: Num rows: 1174353612 Data size: 23487072256 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (ss_sold_date_sk is not null and ss_item_sk is not null) (type: boolean)
                    Statistics: Num rows: 1174353612 Data size: 23383406888 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: ss_sold_date_sk (type: int), ss_item_sk (type: int), ss_ticket_number (type: int), ss_quantity (type: int), ss_ext_sales_price (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                      Statistics: Num rows: 1174353612 Data size: 23383406888 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col0 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col1, _col2, _col3, _col4, _col6
                        input vertices:
                          1 Map 28
                        Statistics: Num rows: 213662719 Data size: 3418603504 Basic stats: COMPLETE Column stats: COMPLETE
                        Reduce Output Operator
                          key expressions: _col2 (type: int), _col1 (type: int)
                          sort order: ++
                          Map-reduce partition columns: _col2 (type: int), _col1 (type: int)
                          Statistics: Num rows: 213662719 Data size: 3418603504 Basic stats: COMPLETE Column stats: COMPLETE
                          value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int)
            Execution mode: vectorized
        Map 28 
            Map Operator Tree:
                TableScan
                  alias: date_dim
                  filterExpr: ((d_year = 2001) and d_date_sk is not null) (type: boolean)
                  Statistics: Num rows: 45363 Data size: 362905 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((d_year = 2001) and d_date_sk is not null) (type: boolean)
                    Statistics: Num rows: 405 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: d_date_sk (type: int)
                      outputColumnNames: _col0
                      Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: 2001 (type: int)
            Execution mode: vectorized
        Map 29 
            Map Operator Tree:
                TableScan
                  alias: store_returns
                  filterExpr: sr_item_sk is not null (type: boolean)
                  Statistics: Num rows: 167243952 Data size: 2675903232 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: sr_item_sk is not null (type: boolean)
                    Statistics: Num rows: 167243952 Data size: 2667828428 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: sr_item_sk (type: int), sr_ticket_number (type: int), sr_return_quantity (type: int), sr_return_amt (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Statistics: Num rows: 167243952 Data size: 2667828428 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col1 (type: int), _col0 (type: int)
                        sort order: ++
                        Map-reduce partition columns: _col1 (type: int), _col0 (type: int)
                        Statistics: Num rows: 167243952 Data size: 2667828428 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col2 (type: int), _col3 (type: float)
            Execution mode: vectorized
        Map 30 
            Map Operator Tree:
                TableScan
                  alias: item
                  filterExpr: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                  Statistics: Num rows: 118835 Data size: 3089722 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                    Statistics: Num rows: 11836 Data size: 1301772 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: i_item_sk (type: int), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int), i_manufact_id (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col5
                      Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col5 (type: int)
            Execution mode: vectorized
        Map 31 
            Map Operator Tree:
                TableScan
                  alias: web_sales
                  filterExpr: (ws_sold_date_sk is not null and ws_item_sk is not null) (type: boolean)
                  Statistics: Num rows: 447759411 Data size: 8955188224 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (ws_sold_date_sk is not null and ws_item_sk is not null) (type: boolean)
                    Statistics: Num rows: 447759411 Data size: 8955044072 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: ws_sold_date_sk (type: int), ws_item_sk (type: int), ws_order_number (type: int), ws_quantity (type: int), ws_ext_sales_price (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                      Statistics: Num rows: 447759411 Data size: 8955044072 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col0 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col1, _col2, _col3, _col4, _col6
                        input vertices:
                          1 Map 33
                        Statistics: Num rows: 81465661 Data size: 1303450576 Basic stats: COMPLETE Column stats: COMPLETE
                        Reduce Output Operator
                          key expressions: _col2 (type: int), _col1 (type: int)
                          sort order: ++
                          Map-reduce partition columns: _col2 (type: int), _col1 (type: int)
                          Statistics: Num rows: 81465661 Data size: 1303450576 Basic stats: COMPLETE Column stats: COMPLETE
                          value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int)
            Execution mode: vectorized
        Map 33 
            Map Operator Tree:
                TableScan
                  alias: date_dim
                  filterExpr: ((d_year = 2001) and d_date_sk is not null) (type: boolean)
                  Statistics: Num rows: 45363 Data size: 362905 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((d_year = 2001) and d_date_sk is not null) (type: boolean)
                    Statistics: Num rows: 405 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: d_date_sk (type: int)
                      outputColumnNames: _col0
                      Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: 2001 (type: int)
            Execution mode: vectorized
        Map 34 
            Map Operator Tree:
                TableScan
                  alias: web_returns
                  filterExpr: wr_item_sk is not null (type: boolean)
                  Statistics: Num rows: 50457044 Data size: 807312704 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: wr_item_sk is not null (type: boolean)
                    Statistics: Num rows: 50457044 Data size: 804725540 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: wr_item_sk (type: int), wr_order_number (type: int), wr_return_quantity (type: int), wr_return_amt (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Statistics: Num rows: 50457044 Data size: 804725540 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col1 (type: int), _col0 (type: int)
                        sort order: ++
                        Map-reduce partition columns: _col1 (type: int), _col0 (type: int)
                        Statistics: Num rows: 50457044 Data size: 804725540 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col2 (type: int), _col3 (type: float)
            Execution mode: vectorized
        Map 35 
            Map Operator Tree:
                TableScan
                  alias: item
                  filterExpr: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                  Filter Operator
                    predicate: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                    Select Operator
                      expressions: i_item_sk (type: int), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int), i_manufact_id (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col5
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col1 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col3, _col4, _col6, _col9, _col10, _col12, _col13, _col14, _col16
                        input vertices:
                          0 Reducer 32
                        Select Operator
                          expressions: _col6 (type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col16 (type: int), (_col3 - COALESCE(_col9,0)) (type: int), (UDFToDouble(_col4) - COALESCE(_col10,0.0)) (type: double)
                          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                          Group By Operator
                            aggregations: sum(_col5), sum(_col6)
                            keys: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                            mode: hash
                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                            Reduce Output Operator
                              key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                              sort order: +++++
                              Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                              value expressions: _col5 (type: bigint), _col6 (type: double)
            Execution mode: vectorized
        Map 6 
            Map Operator Tree:
                TableScan
                  alias: date_dim
                  filterExpr: ((d_year = 2002) and d_date_sk is not null) (type: boolean)
                  Statistics: Num rows: 45363 Data size: 362905 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((d_year = 2002) and d_date_sk is not null) (type: boolean)
                    Statistics: Num rows: 405 Data size: 3240 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: d_date_sk (type: int)
                      outputColumnNames: _col0
                      Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 405 Data size: 1620 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: 2002 (type: int)
            Execution mode: vectorized
        Map 7 
            Map Operator Tree:
                TableScan
                  alias: catalog_returns
                  filterExpr: cr_item_sk is not null (type: boolean)
                  Statistics: Num rows: 108409176 Data size: 1734546816 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: cr_item_sk is not null (type: boolean)
                    Statistics: Num rows: 108409176 Data size: 1729935996 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: cr_item_sk (type: int), cr_order_number (type: int), cr_return_quantity (type: int), cr_return_amount (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Statistics: Num rows: 108409176 Data size: 1729935996 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col1 (type: int), _col0 (type: int)
                        sort order: ++
                        Map-reduce partition columns: _col1 (type: int), _col0 (type: int)
                        Statistics: Num rows: 108409176 Data size: 1729935996 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col2 (type: int), _col3 (type: float)
            Execution mode: vectorized
        Map 8 
            Map Operator Tree:
                TableScan
                  alias: item
                  filterExpr: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                  Statistics: Num rows: 118835 Data size: 3089722 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((((((i_category = 'Sports') and i_item_sk is not null) and i_brand_id is not null) and i_class_id is not null) and i_category_id is not null) and i_manufact_id is not null) (type: boolean)
                    Statistics: Num rows: 11836 Data size: 1301772 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: i_item_sk (type: int), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int), i_manufact_id (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col5
                      Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col0 (type: int)
                        sort order: +
                        Map-reduce partition columns: _col0 (type: int)
                        Statistics: Num rows: 11836 Data size: 236532 Basic stats: COMPLETE Column stats: COMPLETE
                        value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col5 (type: int)
            Execution mode: vectorized
        Map 9 
            Map Operator Tree:
                TableScan
                  alias: store_sales
                  filterExpr: (ss_sold_date_sk is not null and ss_item_sk is not null) (type: boolean)
                  Statistics: Num rows: 1174353612 Data size: 23487072256 Basic stats: COMPLETE Column stats: COMPLETE
                  Filter Operator
                    predicate: (ss_sold_date_sk is not null and ss_item_sk is not null) (type: boolean)
                    Statistics: Num rows: 1174353612 Data size: 23383406888 Basic stats: COMPLETE Column stats: COMPLETE
                    Select Operator
                      expressions: ss_sold_date_sk (type: int), ss_item_sk (type: int), ss_ticket_number (type: int), ss_quantity (type: int), ss_ext_sales_price (type: float)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                      Statistics: Num rows: 1174353612 Data size: 23383406888 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col0 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col1, _col2, _col3, _col4, _col6
                        input vertices:
                          1 Map 11
                        Statistics: Num rows: 213662719 Data size: 3418603504 Basic stats: COMPLETE Column stats: COMPLETE
                        Reduce Output Operator
                          key expressions: _col2 (type: int), _col1 (type: int)
                          sort order: ++
                          Map-reduce partition columns: _col2 (type: int), _col1 (type: int)
                          Statistics: Num rows: 213662719 Data size: 3418603504 Basic stats: COMPLETE Column stats: COMPLETE
                          value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int)
            Execution mode: vectorized
        Reducer 10 
            Reduce Operator Tree:
              Merge Join Operator
                condition map:
                     Left Outer Join0 to 1
                keys:
                  0 _col2 (type: int), _col1 (type: int)
                  1 _col1 (type: int), _col0 (type: int)
                outputColumnNames: _col1, _col3, _col4, _col6, _col9, _col10
                Map Join Operator
                  condition map:
                       Inner Join 0 to 1
                  keys:
                    0 _col1 (type: int)
                    1 _col0 (type: int)
                  outputColumnNames: _col3, _col4, _col6, _col9, _col10, _col12, _col13, _col14, _col16
                  input vertices:
                    1 Map 13
                  Select Operator
                    expressions: _col6 (type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col16 (type: int), (_col3 - COALESCE(_col9,0)) (type: int), (UDFToDouble(_col4) - COALESCE(_col10,0.0)) (type: double)
                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                    Group By Operator
                      aggregations: sum(_col5), sum(_col6)
                      keys: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                      mode: hash
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                      Reduce Output Operator
                        key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        sort order: +++++
                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        value expressions: _col5 (type: bigint), _col6 (type: double)
        Reducer 15 
            Reduce Operator Tree:
              Merge Join Operator
                condition map:
                     Left Outer Join0 to 1
                keys:
                  0 _col2 (type: int), _col1 (type: int)
                  1 _col1 (type: int), _col0 (type: int)
                outputColumnNames: _col1, _col3, _col4, _col6, _col9, _col10
                Statistics: Num rows: 8204 Data size: 164080 Basic stats: COMPLETE Column stats: COMPLETE
                Reduce Output Operator
                  key expressions: _col1 (type: int)
                  sort order: +
                  Map-reduce partition columns: _col1 (type: int)
                  Statistics: Num rows: 8204 Data size: 164080 Basic stats: COMPLETE Column stats: COMPLETE
                  value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int), _col9 (type: int), _col10 (type: float)
        Reducer 2 
            Reduce Operator Tree:
              Merge Join Operator
                condition map:
                     Left Outer Join0 to 1
                keys:
                  0 _col2 (type: int), _col1 (type: int)
                  1 _col1 (type: int), _col0 (type: int)
                outputColumnNames: _col1, _col3, _col4, _col6, _col9, _col10
                Map Join Operator
                  condition map:
                       Inner Join 0 to 1
                  keys:
                    0 _col1 (type: int)
                    1 _col0 (type: int)
                  outputColumnNames: _col3, _col4, _col6, _col9, _col10, _col12, _col13, _col14, _col16
                  input vertices:
                    1 Map 8
                  Select Operator
                    expressions: _col6 (type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col16 (type: int), (_col3 - COALESCE(_col9,0)) (type: int), (UDFToDouble(_col4) - COALESCE(_col10,0.0)) (type: double)
                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                    Group By Operator
                      aggregations: sum(_col5), sum(_col6)
                      keys: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                      mode: hash
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                      Reduce Output Operator
                        key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        sort order: +++++
                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        value expressions: _col5 (type: bigint), _col6 (type: double)
        Reducer 20 
            Reduce Operator Tree:
              Merge Join Operator
                condition map:
                     Left Outer Join0 to 1
                keys:
                  0 _col2 (type: int), _col1 (type: int)
                  1 _col1 (type: int), _col0 (type: int)
                outputColumnNames: _col1, _col3, _col4, _col6, _col9, _col10
                Map Join Operator
                  condition map:
                       Inner Join 0 to 1
                  keys:
                    0 _col1 (type: int)
                    1 _col0 (type: int)
                  outputColumnNames: _col3, _col4, _col6, _col9, _col10, _col12, _col13, _col14, _col16
                  input vertices:
                    1 Map 25
                  Select Operator
                    expressions: _col6 (type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col16 (type: int), (_col3 - COALESCE(_col9,0)) (type: int), (UDFToDouble(_col4) - COALESCE(_col10,0.0)) (type: double)
                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                    Group By Operator
                      aggregations: sum(_col5), sum(_col6)
                      keys: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                      mode: hash
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                      Reduce Output Operator
                        key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        sort order: +++++
                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        value expressions: _col5 (type: bigint), _col6 (type: double)
        Reducer 22 
            Reduce Operator Tree:
              Group By Operator
                aggregations: sum(VALUE._col0), sum(VALUE._col1)
                keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int), KEY._col3 (type: int), KEY._col4 (type: int)
                mode: mergepartial
                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                Reduce Output Operator
                  key expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                  sort order: ++++
                  Map-reduce partition columns: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                  value expressions: _col0 (type: int), _col5 (type: bigint), _col6 (type: double)
        Reducer 27 
            Reduce Operator Tree:
              Merge Join Operator
                condition map:
                     Left Outer Join0 to 1
                keys:
                  0 _col2 (type: int), _col1 (type: int)
                  1 _col1 (type: int), _col0 (type: int)
                outputColumnNames: _col1, _col3, _col4, _col6, _col9, _col10
                Map Join Operator
                  condition map:
                       Inner Join 0 to 1
                  keys:
                    0 _col1 (type: int)
                    1 _col0 (type: int)
                  outputColumnNames: _col3, _col4, _col6, _col9, _col10, _col12, _col13, _col14, _col16
                  input vertices:
                    1 Map 30
                  Select Operator
                    expressions: _col6 (type: int), _col12 (type: int), _col13 (type: int), _col14 (type: int), _col16 (type: int), (_col3 - COALESCE(_col9,0)) (type: int), (UDFToDouble(_col4) - COALESCE(_col10,0.0)) (type: double)
                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                    Group By Operator
                      aggregations: sum(_col5), sum(_col6)
                      keys: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                      mode: hash
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                      Reduce Output Operator
                        key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        sort order: +++++
                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                        value expressions: _col5 (type: bigint), _col6 (type: double)
        Reducer 32 
            Reduce Operator Tree:
              Merge Join Operator
                condition map:
                     Left Outer Join0 to 1
                keys:
                  0 _col2 (type: int), _col1 (type: int)
                  1 _col1 (type: int), _col0 (type: int)
                outputColumnNames: _col1, _col3, _col4, _col6, _col9, _col10
                Statistics: Num rows: 8204 Data size: 164080 Basic stats: COMPLETE Column stats: COMPLETE
                Reduce Output Operator
                  key expressions: _col1 (type: int)
                  sort order: +
                  Map-reduce partition columns: _col1 (type: int)
                  Statistics: Num rows: 8204 Data size: 164080 Basic stats: COMPLETE Column stats: COMPLETE
                  value expressions: _col3 (type: int), _col4 (type: float), _col6 (type: int), _col9 (type: int), _col10 (type: float)
        Reducer 4 
            Reduce Operator Tree:
              Group By Operator
                aggregations: sum(VALUE._col0), sum(VALUE._col1)
                keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int), KEY._col3 (type: int), KEY._col4 (type: int)
                mode: mergepartial
                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                Map Join Operator
                  condition map:
                       Inner Join 0 to 1
                  keys:
                    0 _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                    1 _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int)
                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col12, _col13
                  input vertices:
                    1 Reducer 22
                  Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
                  Filter Operator
                    predicate: ((CAST( _col5 AS decimal(17,2)) / CAST( _col12 AS decimal(17,2))) < CAST( 0.9 AS decimal(37,20))) (type: boolean)
                    Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
                    Select Operator
                      expressions: _col7 (type: int), _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col12 (type: bigint), _col5 (type: bigint), (_col5 - _col12) (type: bigint), (_col6 - _col13) (type: double)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                      Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
                      Reduce Output Operator
                        key expressions: _col8 (type: bigint)
                        sort order: +
                        Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
                        TopN Hash Memory Usage: 0.04
                        value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: bigint), _col7 (type: bigint), _col9 (type: double)
        Reducer 5 
            Reduce Operator Tree:
              Select Operator
                expressions: VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: bigint), VALUE._col7 (type: bigint), KEY.reducesinkkey0 (type: bigint), VALUE._col8 (type: double)
                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
                Limit
                  Number of rows: 100
                  Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
                  File Output Operator
                    compressed: false
                    Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
                    table:
                        input format: org.apache.hadoop.mapred.TextInputFormat
                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
        Union 21 
            Vertex: Union 21
        Union 3 
            Vertex: Union 3

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

Time taken: 12.351 seconds, Fetched: 821 row(s)

{code}



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