Index: jdbc/src/java/org/apache/hadoop/hive/jdbc/HiveDatabaseMetaData.java =================================================================== --- jdbc/src/java/org/apache/hadoop/hive/jdbc/HiveDatabaseMetaData.java (revision 1190431) +++ jdbc/src/java/org/apache/hadoop/hive/jdbc/HiveDatabaseMetaData.java (working copy) @@ -106,7 +106,7 @@ public ResultSet getCatalogs() throws SQLException { try { // TODO a client call to get the schema's after HIVE-675 is implemented - final List catalogs = new ArrayList(); + final List catalogs = new ArrayList(); catalogs.add("default"); return new HiveMetaDataResultSet(Arrays.asList("TABLE_CAT") , Arrays.asList("STRING") @@ -571,7 +571,7 @@ public ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern, String[] types) throws SQLException { final List tablesstr; - final List resultTables = new ArrayList(); + final List resultTables = new ArrayList(); final String resultCatalog; if (catalog==null) { // On jdbc the default catalog is null but on hive it's "default" resultCatalog = "default"; Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java =================================================================== --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 1190431) +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy) @@ -378,6 +378,7 @@ HIVEOPTPPD("hive.optimize.ppd", true), // predicate pushdown HIVEPPDRECOGNIZETRANSITIVITY("hive.ppd.recognizetransivity", true), // predicate pushdown HIVEPPDREMOVEDUPLICATEFILTERS("hive.ppd.remove.duplicatefilters", true), + HIVEMETADATAONLYQUERIES("hive.optimize.metadataonly", true), // push predicates down to storage handlers HIVEOPTPPD_STORAGE("hive.optimize.ppd.storage", true), HIVEOPTGROUPBY("hive.optimize.groupby", true), // optimize group by Index: serde/src/java/org/apache/hadoop/hive/serde2/NullStructSerDe.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/NullStructSerDe.java (revision 0) +++ serde/src/java/org/apache/hadoop/hive/serde2/NullStructSerDe.java (revision 0) @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.serde2; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; + +/** + * Placeholder SerDe for cases where neither serialization nor deserialization is needed + * + */ +public class NullStructSerDe implements SerDe { + + class NullStructField implements StructField { + @Override + public String getFieldName() { + return null; + } + + @Override + public ObjectInspector getFieldObjectInspector() { + return null; + } + + @Override + public String getFieldComment() { + return ""; + } + } + + @Override + public Object deserialize(Writable blob) throws SerDeException { + return null; + } + + @Override + public ObjectInspector getObjectInspector() throws SerDeException { + return new StructObjectInspector() { + public String getTypeName() { + return "null"; + } + public Category getCategory() { + return Category.PRIMITIVE; + } + @Override + public StructField getStructFieldRef(String fieldName) { + return null; + } + @Override + public List getAllStructFieldRefs() { + return new ArrayList(); + } + @Override + public Object getStructFieldData(Object data, StructField fieldRef) { + return null; + } + @Override + public List getStructFieldsDataAsList(Object data) { + return new ArrayList(); + } + }; + } + + @Override + public SerDeStats getSerDeStats() { + return null; + } + + @Override + public void initialize(Configuration conf, Properties tbl) throws SerDeException { + } + + @Override + public Class getSerializedClass() { + return NullWritable.class; + } + + @Override + public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException { + return NullWritable.get(); + } + +} Index: ql/src/test/results/clientpositive/metadataonly1.q.out =================================================================== --- ql/src/test/results/clientpositive/metadataonly1.q.out (revision 0) +++ ql/src/test/results/clientpositive/metadataonly1.q.out (revision 0) @@ -0,0 +1,1518 @@ +PREHOOK: query: CREATE TABLE TEST1(A INT, B DOUBLE) partitioned by (ds string) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE TEST1(A INT, B DOUBLE) partitioned by (ds string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@TEST1 +PREHOOK: query: explain extended select max(ds) from TEST1 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select max(ds) from TEST1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION max (TOK_TABLE_OR_COL ds)))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + test1 + TableScan + alias: test1 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + outputColumnNames: ds + Group By Operator + aggregations: + expr: max(ds) + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: string + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: + expr: max(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-12_585_1868075955473519664/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-12_585_1868075955473519664/-ext-10001/ + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select max(ds) from TEST1 +PREHOOK: type: QUERY +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-12_860_603571457300760090/-mr-10000 +POSTHOOK: query: select max(ds) from TEST1 +POSTHOOK: type: QUERY +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-12_860_603571457300760090/-mr-10000 + +PREHOOK: query: alter table TEST1 add partition (ds='1') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Input: default@test1 +POSTHOOK: query: alter table TEST1 add partition (ds='1') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Input: default@test1 +POSTHOOK: Output: default@test1@ds=1 +PREHOOK: query: explain extended select max(ds) from TEST1 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select max(ds) from TEST1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION max (TOK_TABLE_OR_COL ds)))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + test1 + TableScan + alias: test1 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + outputColumnNames: ds + Group By Operator + aggregations: + expr: max(ds) + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: string + Needs Tagging: false + Path -> Alias: + fake-path-metadata-only-query-default.test1{ds=1} [test1] + Path -> Partition: + fake-path-metadata-only-query-default.test1{ds=1} + Partition + base file name: ds=1 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740998 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: max(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-18_858_3586583087895818520/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-18_858_3586583087895818520/-ext-10001/ + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select max(ds) from TEST1 +PREHOOK: type: QUERY +PREHOOK: Input: default@test1@ds=1 +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-18_974_2551947253137562059/-mr-10000 +POSTHOOK: query: select max(ds) from TEST1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test1@ds=1 +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-18_974_2551947253137562059/-mr-10000 +1 +PREHOOK: query: explain extended select count(distinct ds) from TEST1 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(distinct ds) from TEST1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONDI count (TOK_TABLE_OR_COL ds)))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + test1 + TableScan + alias: test1 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + outputColumnNames: ds + Group By Operator + aggregations: + expr: count(DISTINCT ds) + bucketGroup: false + keys: + expr: ds + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + tag: -1 + value expressions: + expr: _col1 + type: bigint + Needs Tagging: false + Path -> Alias: + fake-path-metadata-only-query-default.test1{ds=1} [test1] + Path -> Partition: + fake-path-metadata-only-query-default.test1{ds=1} + Partition + base file name: ds=1 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740998 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(DISTINCT KEY._col0:0._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: bigint + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-24_614_8262716029839319533/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-24_614_8262716029839319533/-ext-10001/ + 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 count(distinct ds) from TEST1 +PREHOOK: type: QUERY +PREHOOK: Input: default@test1@ds=1 +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-24_727_7465805034346288209/-mr-10000 +POSTHOOK: query: select count(distinct ds) from TEST1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test1@ds=1 +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-24_727_7465805034346288209/-mr-10000 +1 +PREHOOK: query: explain extended select count(ds) from TEST1 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(ds) from TEST1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION count (TOK_TABLE_OR_COL ds)))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + test1 + TableScan + alias: test1 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + outputColumnNames: ds + Group By Operator + aggregations: + expr: count(ds) + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + Needs Tagging: false + Path -> Alias: + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 [test1] + Path -> Partition: + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + Partition + base file name: ds=1 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740998 + 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: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + 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 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-30_281_2767795707655569681/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-30_281_2767795707655569681/-ext-10001/ + 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 count(ds) from TEST1 +PREHOOK: type: QUERY +PREHOOK: Input: default@test1@ds=1 +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-30_392_2741280504992120668/-mr-10000 +POSTHOOK: query: select count(ds) from TEST1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test1@ds=1 +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-30_392_2741280504992120668/-mr-10000 +0 +PREHOOK: query: alter table TEST1 add partition (ds='2') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Input: default@test1 +POSTHOOK: query: alter table TEST1 add partition (ds='2') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Input: default@test1 +POSTHOOK: Output: default@test1@ds=2 +PREHOOK: query: explain extended +select count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME TEST1) a2) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION max (TOK_TABLE_OR_COL ds)) m)))) b) (= (. (TOK_TABLE_OR_COL a2) ds) (. (TOK_TABLE_OR_COL b) m)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTIONSTAR count))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b:test1 + TableScan + alias: test1 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + outputColumnNames: ds + Group By Operator + aggregations: + expr: max(ds) + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: string + Needs Tagging: false + Path -> Alias: + fake-path-metadata-only-query-default.test1{ds=1} [b:test1] + fake-path-metadata-only-query-default.test1{ds=2} [b:test1] + Path -> Partition: + fake-path-metadata-only-query-default.test1{ds=1} + Partition + base file name: ds=1 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740998 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + fake-path-metadata-only-query-default.test1{ds=2} + Partition + base file name: ds=2 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=2 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741016 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: max(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10002 + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + a2 + TableScan + alias: a2 + GatherStats: false + Reduce Output Operator + key expressions: + expr: ds + type: string + sort order: + + Map-reduce partition columns: + expr: ds + type: string + tag: 0 + Needs Tagging: true + Path -> Alias: + file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10002 [$INTNAME] + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 [a2] + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=2 [a2] + Path -> Partition: + file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10002 + Partition + base file name: -mr-10002 + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + Partition + base file name: ds=1 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740998 + 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: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=2 + Partition + base file name: ds=2 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=2 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741016 + 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: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + handleSkewJoin: false + Select Operator + Group By Operator + aggregations: + expr: count() + bucketGroup: false + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10003 + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10003 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + Needs Tagging: false + Path -> Alias: + file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10003 [file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10003] + Path -> Partition: + file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-mr-10003 + Partition + base file name: -mr-10003 + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types bigint + 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 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_629_6256982309049932082/-ext-10001/ + 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 count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m +PREHOOK: type: QUERY +PREHOOK: Input: default@test1@ds=1 +PREHOOK: Input: default@test1@ds=2 +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_904_6796791888883698844/-mr-10000 +POSTHOOK: query: select count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test1@ds=1 +POSTHOOK: Input: default@test1@ds=2 +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-36_904_6796791888883698844/-mr-10000 +0 +PREHOOK: query: CREATE TABLE TEST2(A INT, B DOUBLE) partitioned by (ds string, hr string) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE TEST2(A INT, B DOUBLE) partitioned by (ds string, hr string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@TEST2 +PREHOOK: query: alter table TEST2 add partition (ds='1', hr='1') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Input: default@test2 +POSTHOOK: query: alter table TEST2 add partition (ds='1', hr='1') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Input: default@test2 +POSTHOOK: Output: default@test2@ds=1/hr=1 +PREHOOK: query: alter table TEST2 add partition (ds='1', hr='2') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Input: default@test2 +POSTHOOK: query: alter table TEST2 add partition (ds='1', hr='2') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Input: default@test2 +POSTHOOK: Output: default@test2@ds=1/hr=2 +PREHOOK: query: alter table TEST2 add partition (ds='1', hr='3') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Input: default@test2 +POSTHOOK: query: alter table TEST2 add partition (ds='1', hr='3') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Input: default@test2 +POSTHOOK: Output: default@test2@ds=1/hr=3 +PREHOOK: query: explain extended select ds, count(distinct hr) from TEST2 group by ds +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select ds, count(distinct hr) from TEST2 group by ds +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST2))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL ds)) (TOK_SELEXPR (TOK_FUNCTIONDI count (TOK_TABLE_OR_COL hr)))) (TOK_GROUPBY (TOK_TABLE_OR_COL ds)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + test2 + TableScan + alias: test2 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + expr: hr + type: string + outputColumnNames: ds, hr + Group By Operator + aggregations: + expr: count(DISTINCT hr) + bucketGroup: false + keys: + expr: ds + type: string + expr: hr + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Needs Tagging: false + Path -> Alias: + fake-path-metadata-only-query-default.test2{ds=1, hr=1} [test2] + fake-path-metadata-only-query-default.test2{ds=1, hr=2} [test2] + fake-path-metadata-only-query-default.test2{ds=1, hr=3} [test2] + Path -> Partition: + fake-path-metadata-only-query-default.test2{ds=1, hr=1} + Partition + base file name: hr=1 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + hr 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=1 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test2 + name: default.test2 + fake-path-metadata-only-query-default.test2{ds=1, hr=2} + Partition + base file name: hr=2 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + hr 2 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test2 + name: default.test2 + fake-path-metadata-only-query-default.test2{ds=1, hr=3} + Partition + base file name: hr=3 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + hr 3 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=3 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test2 + name: default.test2 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(DISTINCT KEY._col1:0._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-54_867_6911418128650417746/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-54_867_6911418128650417746/-ext-10001/ + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select ds, count(distinct hr) from TEST2 group by ds +PREHOOK: type: QUERY +PREHOOK: Input: default@test2@ds=1/hr=1 +PREHOOK: Input: default@test2@ds=1/hr=2 +PREHOOK: Input: default@test2@ds=1/hr=3 +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-55_001_7910447671567792543/-mr-10000 +POSTHOOK: query: select ds, count(distinct hr) from TEST2 group by ds +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test2@ds=1/hr=1 +POSTHOOK: Input: default@test2@ds=1/hr=2 +POSTHOOK: Input: default@test2@ds=1/hr=3 +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-43-55_001_7910447671567792543/-mr-10000 +1 3 +PREHOOK: query: explain extended select ds, count(hr) from TEST2 group by ds +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select ds, count(hr) from TEST2 group by ds +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST2))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL ds)) (TOK_SELEXPR (TOK_FUNCTION count (TOK_TABLE_OR_COL hr)))) (TOK_GROUPBY (TOK_TABLE_OR_COL ds)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + test2 + TableScan + alias: test2 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + expr: hr + type: string + outputColumnNames: ds, hr + Group By Operator + aggregations: + expr: count(hr) + bucketGroup: false + keys: + expr: ds + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Needs Tagging: false + Path -> Alias: + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=1 [test2] + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=2 [test2] + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=3 [test2] + Path -> Partition: + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=1 + Partition + base file name: hr=1 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + hr 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=1 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + 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: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test2 + name: default.test2 + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=2 + Partition + base file name: hr=2 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + hr 2 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + 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: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test2 + name: default.test2 + pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=3 + Partition + base file name: hr=3 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + hr 3 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2/ds=1/hr=3 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + 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: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test2 + name default.test2 + partition_columns ds/hr + serialization.ddl struct test2 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741034 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test2 + name: default.test2 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-00_613_3770359973599871660/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-00_613_3770359973599871660/-ext-10001/ + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1 + columns.types string:bigint + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select ds, count(hr) from TEST2 group by ds +PREHOOK: type: QUERY +PREHOOK: Input: default@test2@ds=1/hr=1 +PREHOOK: Input: default@test2@ds=1/hr=2 +PREHOOK: Input: default@test2@ds=1/hr=3 +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-00_729_5714201473330188446/-mr-10000 +POSTHOOK: query: select ds, count(hr) from TEST2 group by ds +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test2@ds=1/hr=1 +POSTHOOK: Input: default@test2@ds=1/hr=2 +POSTHOOK: Input: default@test2@ds=1/hr=3 +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-00_729_5714201473330188446/-mr-10000 +PREHOOK: query: explain extended select max(ds) from TEST1 +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select max(ds) from TEST1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME TEST1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION max (TOK_TABLE_OR_COL ds)))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + test1 + TableScan + alias: test1 + GatherStats: false + Select Operator + expressions: + expr: ds + type: string + outputColumnNames: ds + Group By Operator + aggregations: + expr: max(ds) + bucketGroup: false + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: string + Needs Tagging: false + Path -> Alias: + fake-path-metadata-only-query-default.test1{ds=1} [test1] + fake-path-metadata-only-query-default.test1{ds=2} [test1] + Path -> Partition: + fake-path-metadata-only-query-default.test1{ds=1} + Partition + base file name: ds=1 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 1 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740998 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + fake-path-metadata-only-query-default.test1{ds=2} + Partition + base file name: ds=2 + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2 + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1/ds=2 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319741016 + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count -1 + columns a,b + columns.types int:double + file.inputformat org.apache.hadoop.mapred.TextInputFormat + file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + location pfile:/Users/njain/hive/hive2/build/ql/test/data/warehouse/test1 + name default.test1 + partition_columns ds + serialization.ddl struct test1 { i32 a, double b} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + transient_lastDdlTime 1319740992 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test1 + name: default.test1 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: max(VALUE._col0) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + directory: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-06_562_9209841409592787703/-ext-10001 + NumFilesPerFileSink: 1 + Stats Publishing Key Prefix: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-06_562_9209841409592787703/-ext-10001/ + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + serialization.format 1 + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select max(ds) from TEST1 +PREHOOK: type: QUERY +PREHOOK: Input: default@test1@ds=1 +PREHOOK: Input: default@test1@ds=2 +PREHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-06_683_5969170878727093036/-mr-10000 +POSTHOOK: query: select max(ds) from TEST1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test1@ds=1 +POSTHOOK: Input: default@test1@ds=2 +POSTHOOK: Output: file:/var/folders/bZ/bZe+iKfoFTuPoShRd6dy6-tOU9Y/-Tmp-/njain/hive_2011-10-27_11-44-06_683_5969170878727093036/-mr-10000 +2 Index: ql/src/test/queries/clientpositive/metadataonly1.q =================================================================== --- ql/src/test/queries/clientpositive/metadataonly1.q (revision 0) +++ ql/src/test/queries/clientpositive/metadataonly1.q (revision 0) @@ -0,0 +1,35 @@ +CREATE TABLE TEST1(A INT, B DOUBLE) partitioned by (ds string); +explain extended select max(ds) from TEST1; +select max(ds) from TEST1; + +alter table TEST1 add partition (ds='1'); +explain extended select max(ds) from TEST1; +select max(ds) from TEST1; + +explain extended select count(distinct ds) from TEST1; +select count(distinct ds) from TEST1; + +explain extended select count(ds) from TEST1; +select count(ds) from TEST1; + +alter table TEST1 add partition (ds='2'); +explain extended +select count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m; +select count(*) from TEST1 a2 join (select max(ds) m from TEST1) b on a2.ds=b.m; + + +CREATE TABLE TEST2(A INT, B DOUBLE) partitioned by (ds string, hr string); +alter table TEST2 add partition (ds='1', hr='1'); +alter table TEST2 add partition (ds='1', hr='2'); +alter table TEST2 add partition (ds='1', hr='3'); + +explain extended select ds, count(distinct hr) from TEST2 group by ds; +select ds, count(distinct hr) from TEST2 group by ds; + +explain extended select ds, count(hr) from TEST2 group by ds; +select ds, count(hr) from TEST2 group by ds; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +explain extended select max(ds) from TEST1; +select max(ds) from TEST1; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java (working copy) @@ -56,6 +56,9 @@ resolvers.add(new IndexWhereResolver()); } resolvers.add(new MapJoinResolver()); + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES)) { + resolvers.add(new MetadataOnlyOptimizer()); + } } /** Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MetadataOnlyOptimizer.java (revision 0) @@ -0,0 +1,284 @@ +package org.apache.hadoop.hive.ql.optimizer.physical; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; +import java.util.Iterator; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; +import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; +import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.GraphWalker; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.lib.PreOrderWalker; +import org.apache.hadoop.hive.ql.lib.Rule; +import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.io.OneNullRowInputFormat; +import org.apache.hadoop.hive.serde2.NullStructSerDe; + +/** + * + * MetadataOnlyOptimizer determines to which TableScanOperators "metadata only" + * optimization can be applied. Such operator must use only partition columns + * (it is easy to check, because we are after column pruning and all places + * where the data from the operator is used must go through GroupByOperator + * distinct or distinct-like aggregations. Aggregation is distinct-like if + * adding distinct wouldn't change the result, for example min, max. + * + * We cannot apply the optimization without group by, because the results depend + * on the numbers of rows in partitions, for example count(hr) will count all + * rows in matching partitions. + * + */ +public class MetadataOnlyOptimizer implements PhysicalPlanResolver { + private static final Log LOG = LogFactory.getLog(MetadataOnlyOptimizer.class.getName()); + + static private class WalkerCtx implements NodeProcessorCtx { + /* operators for which there is chance the optimization can be applied */ + private final HashSet possible = new HashSet(); + /* operators for which the optimization will be successful */ + private final HashSet success = new HashSet(); + + /** + * Sets operator as one for which there is a chance to apply optimization + * + * @param op + * the operator + */ + public void setMayBeMetadataOnly(TableScanOperator op) { + possible.add(op); + } + + /** Convert all possible operators to success */ + public void convertMetadataOnly() { + success.addAll(possible); + possible.clear(); + } + + /** + * Convert all possible operators to banned + */ + public void convertNotMetadataOnly() { + possible.clear(); + success.clear(); + } + + /** + * Returns HashSet of collected operators for which the optimization may be + * applicable. + */ + public HashSet getMayBeMetadataOnlyTableScans() { + return possible; + } + + /** + * Returns HashSet of collected operators for which the optimization is + * applicable. + */ + public HashSet getMetadataOnlyTableScans() { + return success; + } + + } + + static private class TableScanProcessor implements NodeProcessor { + public TableScanProcessor() { + } + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + TableScanOperator node = (TableScanOperator) nd; + WalkerCtx walkerCtx = (WalkerCtx) procCtx; + if (((node.getNeededColumnIDs() == null) || (node.getNeededColumnIDs().size() == 0)) + && ((node.getConf() == null) || + (node.getConf().getVirtualCols() == null) || + (node.getConf().getVirtualCols().isEmpty()))) { + walkerCtx.setMayBeMetadataOnly(node); + } + return nd; + } + } + + static private class FileSinkProcessor implements NodeProcessor { + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + WalkerCtx walkerCtx = (WalkerCtx) procCtx; + // There can be atmost one element eligible to be converted to + // metadata only + if ((walkerCtx.getMayBeMetadataOnlyTableScans().isEmpty()) + || (walkerCtx.getMayBeMetadataOnlyTableScans().size() > 1)) { + return nd; + } + + for (Node op : stack) { + if (op instanceof GroupByOperator) { + GroupByOperator gby = (GroupByOperator) op; + if (!gby.getConf().isDistinctLike()) { + // GroupBy not distinct like, disabling + walkerCtx.convertNotMetadataOnly(); + return nd; + } + } + } + + walkerCtx.convertMetadataOnly(); + return nd; + } + } + + @Override + public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { + Dispatcher disp = new MetadataOnlyTaskDispatcher(pctx); + GraphWalker ogw = new DefaultGraphWalker(disp); + ArrayList topNodes = new ArrayList(); + topNodes.addAll(pctx.rootTasks); + ogw.startWalking(topNodes, null); + return pctx; + } + + /** + * Iterate over all tasks one-to-one and convert them to metadata only + */ + class MetadataOnlyTaskDispatcher implements Dispatcher { + + private PhysicalContext physicalContext; + + public MetadataOnlyTaskDispatcher(PhysicalContext context) { + super(); + physicalContext = context; + } + + private String getAliasForTableScanOperator(MapredWork work, + TableScanOperator tso) { + + for (Map.Entry> entry : work.getAliasToWork().entrySet()) { + if (entry.getValue() == tso) { + return entry.getKey(); + } + } + + return null; + } + + private PartitionDesc changePartitionToMetadataOnly(PartitionDesc desc) { + if (desc != null) { + desc.setInputFileFormatClass(OneNullRowInputFormat.class); + desc.setDeserializerClass(NullStructSerDe.class); + desc.setSerdeClassName(NullStructSerDe.class.getName()); + } + return desc; + } + + private List getPathsForAlias(MapredWork work, String alias) { + List paths = new ArrayList(); + + for (Map.Entry> entry : work.getPathToAliases().entrySet()) { + if (entry.getValue().contains(alias)) { + paths.add(entry.getKey()); + } + } + + return paths; + } + + private void processAlias(MapredWork work, String alias) { + // Change the alias partition desc + PartitionDesc aliasPartn = work.getAliasToPartnInfo().get(alias); + changePartitionToMetadataOnly(aliasPartn); + + List paths = getPathsForAlias(work, alias); + for (String path : paths) { + PartitionDesc newPartition = changePartitionToMetadataOnly(work.getPathToPartitionInfo().get( + path)); + Path fakePath = new Path("file", null, + "/fake-path-metadata-only-query-" + newPartition.getTableName() + + newPartition.getPartSpec().toString()); + work.getPathToPartitionInfo().remove(path); + work.getPathToPartitionInfo().put(fakePath.getName(), newPartition); + ArrayList aliases = work.getPathToAliases().remove(path); + work.getPathToAliases().put(fakePath.getName(), aliases); + } + } + + private void convertToMetadataOnlyQuery(MapredWork work, + TableScanOperator tso) { + String alias = getAliasForTableScanOperator(work, tso); + processAlias(work, alias); + } + + @Override + public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) + throws SemanticException { + Task task = (Task) nd; + + Collection> topOperators + = task.getTopOperators(); + if (topOperators.size() == 0) { + return null; + } + + LOG.info("Looking for table scans where optimization is applicable"); + // create a the context for walking operators + ParseContext parseContext = physicalContext.getParseContext(); + WalkerCtx walkerCtx = new WalkerCtx(); + + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("R1", "TS%"), new TableScanProcessor()); + opRules.put(new RuleRegExp("R2", "GBY%.*FS%"), new FileSinkProcessor()); + + // The dispatcher fires the processor corresponding to the closest + // matching rule and passes the context along + Dispatcher disp = new DefaultRuleDispatcher(null, opRules, walkerCtx); + GraphWalker ogw = new PreOrderWalker(disp); + + // Create a list of topOp nodes + ArrayList topNodes = new ArrayList(); + // Get the top Nodes for this map-reduce task + for (Operator + workOperator : topOperators) { + if (parseContext.getTopOps().values().contains(workOperator)) { + topNodes.add(workOperator); + } + } + + if (task.getReducer() != null) { + topNodes.add(task.getReducer()); + } + + ogw.startWalking(topNodes, null); + + LOG.info(String.format("Found %d metadata only table scans", + walkerCtx.getMetadataOnlyTableScans().size())); + Iterator iterator + = walkerCtx.getMetadataOnlyTableScans().iterator(); + + while (iterator.hasNext()) { + TableScanOperator tso = iterator.next(); + LOG.info("Metadata only table scan for " + tso.getConf().getAlias()); + convertToMetadataOnlyQuery((MapredWork) task.getWork(), tso); + } + + return null; + } + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java (working copy) @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -348,10 +349,18 @@ return false; } + public Collection> getTopOperators() { + return new LinkedList>(); + } + public boolean hasReduce() { return false; } + public Operator getReducer() { + return null; + } + public HashMap getCounters() { return taskCounters; } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java (working copy) @@ -488,4 +488,9 @@ return null; } + + @Override + public Operator getReducer() { + return getWork().getReducer(); + } } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java (working copy) @@ -26,6 +26,7 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Calendar; +import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; @@ -452,6 +453,11 @@ } @Override + public Collection> getTopOperators() { + return getWork().getAliasToWork().values(); + } + + @Override public String getName() { return "MAPREDLOCAL"; } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (working copy) @@ -29,6 +29,7 @@ import java.net.URLDecoder; import java.net.URLEncoder; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Enumeration; import java.util.HashSet; @@ -770,6 +771,11 @@ } @Override + public Collection> getTopOperators() { + return getWork().getAliasToWork().values(); + } + + @Override public boolean hasReduce() { MapredWork w = getWork(); return w.getReducer() != null; Index: ql/src/java/org/apache/hadoop/hive/ql/plan/GroupByDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/GroupByDesc.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/GroupByDesc.java (working copy) @@ -18,6 +18,11 @@ package org.apache.hadoop.hive.ql.plan; +import java.util.ArrayList; + +import org.apache.hadoop.hive.ql.udf.UDFType; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; + /** * GroupByDesc. * @@ -175,4 +180,22 @@ public void setBucketGroup(boolean dataSorted) { bucketGroup = dataSorted; } + + /** + * Checks if this grouping is like distinct, which means that all non-distinct grouping + * columns behave like they were distinct - for example min and max operators. + */ + public boolean isDistinctLike() { + ArrayList aggregators = getAggregators(); + for(AggregationDesc ad: aggregators){ + if(!ad.getDistinct()) { + GenericUDAFEvaluator udafEval = ad.getGenericUDAFEvaluator(); + UDFType annot = udafEval.getClass().getAnnotation(UDFType.class); + if(annot == null || !annot.distinctLike()) { + return false; + } + } + } + return true; + } } Index: ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/io/OneNullRowInputFormat.java (revision 0) @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; + +/** + * OneNullRowInputFormat outputs one null row. Used in implementation of + * metadata only queries. + * + */ +public class OneNullRowInputFormat implements + InputFormat, JobConfigurable { + private static final Log LOG = LogFactory.getLog(OneNullRowInputFormat.class + .getName()); + MapredWork mrwork = null; + List partitions; + long len; + + static public class DummyInputSplit implements InputSplit { + public DummyInputSplit() { + } + + @Override + public long getLength() throws IOException { + return 1; + } + + @Override + public String[] getLocations() throws IOException { + return new String[0]; + } + + @Override + public void readFields(DataInput arg0) throws IOException { + } + + @Override + public void write(DataOutput arg0) throws IOException { + } + + } + + static public class OneNullRowRecordReader implements RecordReader { + private boolean processed = false; + public OneNullRowRecordReader() { + } + @Override + public void close() throws IOException { + } + + @Override + public NullWritable createKey() { + return NullWritable.get(); + } + + @Override + public NullWritable createValue() { + return NullWritable.get(); + } + + @Override + public long getPos() throws IOException { + return (processed ? 1 : 0); + } + + @Override + public float getProgress() throws IOException { + return (float) (processed ? 1.0 : 0.0); + } + + @Override + public boolean next(NullWritable arg0, NullWritable arg1) throws IOException { + if(processed) { + return false; + } else { + processed = true; + return true; + } + } + + } + + @Override + public RecordReader getRecordReader(InputSplit arg0, JobConf arg1, Reporter arg2) + throws IOException { + return new OneNullRowRecordReader(); + } + + @Override + public InputSplit[] getSplits(JobConf arg0, int arg1) throws IOException { + InputSplit[] ret = new InputSplit[1]; + ret[0] = new DummyInputSplit(); + LOG.info("Calculating splits"); + return ret; + } + + @Override + public void configure(JobConf job) { + LOG.info("Using one null row input format"); + } + +} \ No newline at end of file Index: ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java (working copy) @@ -33,4 +33,5 @@ public @interface UDFType { boolean deterministic() default true; boolean stateful() default false; + boolean distinctLike() default false; } Index: ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMax.java (working copy) @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.UDFType; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; @@ -49,6 +50,7 @@ return new GenericUDAFMaxEvaluator(); } + @UDFType(distinctLike=true) public static class GenericUDAFMaxEvaluator extends GenericUDAFEvaluator { ObjectInspector inputOI; Index: ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java (revision 1190431) +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMin.java (working copy) @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.UDFType; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; @@ -49,6 +50,7 @@ return new GenericUDAFMinEvaluator(); } + @UDFType(distinctLike=true) public static class GenericUDAFMinEvaluator extends GenericUDAFEvaluator { ObjectInspector inputOI;