hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From c..@apache.org
Subject svn commit: r1382098 [2/2] - in /hive/trunk: ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ ql/src/java/org/apache/hadoop/hive/ql/plan/ ql/src/java/org/apache/hadoop/hive/ql/util/ ql/src/test/queries/client...
Date Fri, 07 Sep 2012 17:40:15 GMT
Added: hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out?rev=1382098&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out Fri Sep  7 17:40:14
2012
@@ -0,0 +1,567 @@
+PREHOOK: query: -- small no part, 4 bucket & big 2 part, 2 bucket
+CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO
4 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- small no part, 4 bucket & big 2 part, 2 bucket
+CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO
4 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@bucket_small
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE
bucket_small
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_small
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE
bucket_small
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_small
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE
bucket_small
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_small
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE
bucket_small
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_small
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE
bucket_small
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_small
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE
bucket_small
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_small
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE
bucket_small
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_small
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE
bucket_small
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_small
+PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string)
CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string)
CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@bucket_big
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-08')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_big
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-08')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_big
+POSTHOOK: Output: default@bucket_big@ds=2008-04-08
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-08')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_big@ds=2008-04-08
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-08')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_big@ds=2008-04-08
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-09')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_big
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-09')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_big
+POSTHOOK: Output: default@bucket_big@ds=2008-04-09
+PREHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-09')
+PREHOOK: type: LOAD
+PREHOOK: Output: default@bucket_big@ds=2008-04-09
+POSTHOOK: query: load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE
bucket_big partition(ds='2008-04-09')
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@bucket_big@ds=2008-04-09
+PREHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN
bucket_big b ON a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a
JOIN bucket_big b ON a.key = b.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME
bucket_big) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT
(TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST
a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count)))))
+
+STAGE DEPENDENCIES:
+  Stage-4 is a root stage
+  Stage-1 depends on stages: Stage-4
+  Stage-2 depends on stages: Stage-1
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-4
+    Map Reduce Local Work
+      Alias -> Map Local Tables:
+        a 
+          Fetch Operator
+            limit: -1
+      Alias -> Map Local Operator Tree:
+        a 
+          TableScan
+            alias: a
+            GatherStats: false
+            HashTable Sink Operator
+              condition expressions:
+                0 
+                1 
+              handleSkewJoin: false
+              keys:
+                0 [Column[key]]
+                1 [Column[key]]
+              Position of Big Table: 1
+      Bucket Mapjoin Context:
+          Alias Bucket Base File Name Mapping:
+            a {ds=2008-04-08/srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt],
ds=2008-04-08/srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt],
ds=2008-04-09/srcsortbucket1outof4.txt=[srcsortbucket1outof4.txt, srcsortbucket3outof4.txt],
ds=2008-04-09/srcsortbucket2outof4.txt=[srcsortbucket2outof4.txt, srcsortbucket4outof4.txt]}
+          Alias Bucket File Name Mapping:
+#### A masked pattern was here ####
+          Alias Bucket Output File Name Mapping:
+#### A masked pattern was here ####
+
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        b 
+          TableScan
+            alias: b
+            GatherStats: false
+            Map Join Operator
+              condition map:
+                   Inner Join 0 to 1
+              condition expressions:
+                0 
+                1 
+              handleSkewJoin: false
+              keys:
+                0 [Column[key]]
+                1 [Column[key]]
+              Position of Big Table: 1
+              File Output Operator
+                compressed: false
+                GlobalTableId: 0
+#### A masked pattern was here ####
+                NumFilesPerFileSink: 1
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    properties:
+                      columns 
+                      columns.types 
+                      escape.delim \
+                TotalFiles: 1
+                GatherStats: false
+                MultiFileSpray: false
+      Local Work:
+        Map Reduce Local Work
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: ds=2008-04-08
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            partition values:
+              ds 2008-04-08
+            properties:
+              SORTBUCKETCOLSPREFIX TRUE
+              bucket_count 2
+              bucket_field_name key
+              columns key,value
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.bucket_big
+              numFiles 2
+              numPartitions 2
+              numRows 0
+              partition_columns ds
+              rawDataSize 0
+              serialization.ddl struct bucket_big { string key, string value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 2750
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                SORTBUCKETCOLSPREFIX TRUE
+                bucket_count 2
+                bucket_field_name key
+                columns key,value
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.bucket_big
+                numFiles 4
+                numPartitions 2
+                numRows 0
+                partition_columns ds
+                rawDataSize 0
+                serialization.ddl struct bucket_big { string key, string value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 5500
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucket_big
+            name: default.bucket_big
+#### A masked pattern was here ####
+          Partition
+            base file name: ds=2008-04-09
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            partition values:
+              ds 2008-04-09
+            properties:
+              SORTBUCKETCOLSPREFIX TRUE
+              bucket_count 2
+              bucket_field_name key
+              columns key,value
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.bucket_big
+              numFiles 2
+              numPartitions 2
+              numRows 0
+              partition_columns ds
+              rawDataSize 0
+              serialization.ddl struct bucket_big { string key, string value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 2750
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                SORTBUCKETCOLSPREFIX TRUE
+                bucket_count 2
+                bucket_field_name key
+                columns key,value
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.bucket_big
+                numFiles 4
+                numPartitions 2
+                numRows 0
+                partition_columns ds
+                rawDataSize 0
+                serialization.ddl struct bucket_big { string key, string value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 5500
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucket_big
+            name: default.bucket_big
+
+  Stage: Stage-2
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+          Select Operator
+            Select Operator
+              Group By Operator
+                aggregations:
+                      expr: count()
+                bucketGroup: false
+                mode: hash
+                outputColumnNames: _col0
+                Reduce Output Operator
+                  sort order: 
+                  tag: -1
+                  value expressions:
+                        expr: _col0
+                        type: bigint
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -mr-10002
+            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+            properties:
+              columns 
+              columns.types 
+              escape.delim \
+          
+              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+              properties:
+                columns 
+                columns.types 
+                escape.delim \
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+#### A masked pattern was here ####
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    columns _col0
+                    columns.types bigint
+                    escape.delim \
+                    serialization.format 1
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b
ON a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@bucket_big@ds=2008-04-08
+PREHOOK: Input: default@bucket_big@ds=2008-04-09
+PREHOOK: Input: default@bucket_small
+#### A masked pattern was here ####
+POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b
ON a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@bucket_big@ds=2008-04-08
+POSTHOOK: Input: default@bucket_big@ds=2008-04-09
+POSTHOOK: Input: default@bucket_small
+#### A masked pattern was here ####
+928
+PREHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN
bucket_big b ON a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a
JOIN bucket_big b ON a.key = b.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME bucket_small) a) (TOK_TABREF (TOK_TABNAME
bucket_big) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT
(TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST
a))) (TOK_SELEXPR (TOK_FUNCTIONSTAR count)))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        b 
+          TableScan
+            alias: b
+            GatherStats: false
+            Sorted Merge Bucket Map Join Operator
+              condition map:
+                   Inner Join 0 to 1
+              condition expressions:
+                0 
+                1 
+              handleSkewJoin: false
+              keys:
+                0 [Column[key]]
+                1 [Column[key]]
+              Position of Big Table: 1
+              File Output Operator
+                compressed: false
+                GlobalTableId: 0
+#### A masked pattern was here ####
+                NumFilesPerFileSink: 1
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    properties:
+                      columns 
+                      columns.types 
+                      escape.delim \
+                TotalFiles: 1
+                GatherStats: false
+                MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: ds=2008-04-08
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            partition values:
+              ds 2008-04-08
+            properties:
+              SORTBUCKETCOLSPREFIX TRUE
+              bucket_count 2
+              bucket_field_name key
+              columns key,value
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.bucket_big
+              numFiles 2
+              numPartitions 2
+              numRows 0
+              partition_columns ds
+              rawDataSize 0
+              serialization.ddl struct bucket_big { string key, string value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 2750
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                SORTBUCKETCOLSPREFIX TRUE
+                bucket_count 2
+                bucket_field_name key
+                columns key,value
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.bucket_big
+                numFiles 4
+                numPartitions 2
+                numRows 0
+                partition_columns ds
+                rawDataSize 0
+                serialization.ddl struct bucket_big { string key, string value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 5500
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucket_big
+            name: default.bucket_big
+#### A masked pattern was here ####
+          Partition
+            base file name: ds=2008-04-09
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            partition values:
+              ds 2008-04-09
+            properties:
+              SORTBUCKETCOLSPREFIX TRUE
+              bucket_count 2
+              bucket_field_name key
+              columns key,value
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.bucket_big
+              numFiles 2
+              numPartitions 2
+              numRows 0
+              partition_columns ds
+              rawDataSize 0
+              serialization.ddl struct bucket_big { string key, string value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 2750
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                SORTBUCKETCOLSPREFIX TRUE
+                bucket_count 2
+                bucket_field_name key
+                columns key,value
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.bucket_big
+                numFiles 4
+                numPartitions 2
+                numRows 0
+                partition_columns ds
+                rawDataSize 0
+                serialization.ddl struct bucket_big { string key, string value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 5500
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucket_big
+            name: default.bucket_big
+
+  Stage: Stage-2
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+          Select Operator
+            Select Operator
+              Group By Operator
+                aggregations:
+                      expr: count()
+                bucketGroup: false
+                mode: hash
+                outputColumnNames: _col0
+                Reduce Output Operator
+                  sort order: 
+                  tag: -1
+                  value expressions:
+                        expr: _col0
+                        type: bigint
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -mr-10002
+            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+            properties:
+              columns 
+              columns.types 
+              escape.delim \
+          
+              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+              properties:
+                columns 
+                columns.types 
+                escape.delim \
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+#### A masked pattern was here ####
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    columns _col0
+                    columns.types bigint
+                    escape.delim \
+                    serialization.format 1
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
+PREHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b
ON a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@bucket_big@ds=2008-04-08
+PREHOOK: Input: default@bucket_big@ds=2008-04-09
+PREHOOK: Input: default@bucket_small
+#### A masked pattern was here ####
+POSTHOOK: query: select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b
ON a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@bucket_big@ds=2008-04-08
+POSTHOOK: Input: default@bucket_big@ds=2008-04-09
+POSTHOOK: Input: default@bucket_small
+#### A masked pattern was here ####
+928

Modified: hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java?rev=1382098&r1=1382097&r2=1382098&view=diff
==============================================================================
--- hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java (original)
+++ hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java Fri
Sep  7 17:40:14 2012
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.serde2;
 
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
@@ -36,7 +37,7 @@ public final class ColumnProjectionUtils
    * is included in the list, RCFile's reader will not skip its value.
    * 
    */
-  public static void setReadColumnIDs(Configuration conf, ArrayList<Integer> ids) {
+  public static void setReadColumnIDs(Configuration conf, List<Integer> ids) {
     String id = toReadColumnIDString(ids);
     setReadColumnIDConf(conf, id);
   }
@@ -46,8 +47,7 @@ public final class ColumnProjectionUtils
    * is included in the list, RCFile's reader will not skip its value.
    * 
    */
-  public static void appendReadColumnIDs(Configuration conf,
-      ArrayList<Integer> ids) {
+  public static void appendReadColumnIDs(Configuration conf, List<Integer> ids) {
     String id = toReadColumnIDString(ids);
     if (id != null) {
       String old = conf.get(READ_COLUMN_IDS_CONF_STR, null);
@@ -69,7 +69,7 @@ public final class ColumnProjectionUtils
     conf.set(READ_COLUMN_IDS_CONF_STR, id);
   }
 
-  private static String toReadColumnIDString(ArrayList<Integer> ids) {
+  private static String toReadColumnIDString(List<Integer> ids) {
     String id = null;
     if (ids != null) {
       for (int i = 0; i < ids.size(); i++) {



Mime
View raw message