You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "Hari Sankar Sivarama Subramaniyan (JIRA)" <ji...@apache.org> on 2014/03/12 23:16:43 UTC

[jira] [Created] (HIVE-6642) Query fails to vectorize when a non string partition column is part of the query expression

Hari Sankar Sivarama Subramaniyan created HIVE-6642:
-------------------------------------------------------

             Summary: Query fails to vectorize when a non string partition column is part of the query expression
                 Key: HIVE-6642
                 URL: https://issues.apache.org/jira/browse/HIVE-6642
             Project: Hive
          Issue Type: Bug
            Reporter: Hari Sankar Sivarama Subramaniyan
            Assignee: Hari Sankar Sivarama Subramaniyan


drop table if exists alltypesorc_part;

CREATE TABLE alltypesorc_part (
ctinyint tinyint,
csmallint smallint,
cint int,
cbigint bigint,
cfloat float,
cdouble double,
cstring1 string,
cstring2 string,
ctimestamp1 timestamp,
ctimestamp2 timestamp,
cboolean1 boolean,
cboolean2 boolean) partitioned by (ds int) STORED AS ORC;

insert overwrite table alltypesorc_part partition (ds=2011) select * from alltypesorc limit 100;
insert overwrite table alltypesorc_part partition (ds=2012) select * from alltypesorc limit 200;

explain select *
from (select ds from alltypesorc_part) t1,
     alltypesorc t2
where t1.ds = t2.cint
order by t2.ctimestamp1
limit 100;

The above query fails to vectorize because (select ds from alltypesorc_part) t1 returns a string column and the join equality on t2 is performed on an int column. The correct output when vectorization is turned on should be:
STAGE DEPENDENCIES:
  Stage-5 is a root stage
  Stage-2 depends on stages: Stage-5
  Stage-0 is a root stage

STAGE PLANS:
  Stage: Stage-5
    Map Reduce Local Work
      Alias -> Map Local Tables:
        t1:alltypesorc_part
          Fetch Operator
            limit: -1
      Alias -> Map Local Operator Tree:
        t1:alltypesorc_part
          TableScan
            alias: alltypesorc_part
            Statistics: Num rows: 300 Data size: 62328 Basic stats: COMPLETE Column stats: COMPLETE
            Select Operator
              expressions: ds (type: int)
              outputColumnNames: _col0
              Statistics: Num rows: 300 Data size: 1200 Basic stats: COMPLETE Column stats: COMPLETE
              HashTable Sink Operator
                condition expressions:
                  0 {_col0}
                  1 {ctinyint} {csmallint} {cint} {cbigint} {cfloat} {cdouble} {cstring1} {cstring2} {ctimestamp1} {ctimestamp2} {cboolean1} {cboolean2}
                keys:
                  0 _col0 (type: int)
                  1 cint (type: int)

  Stage: Stage-2
    Map Reduce
      Map Operator Tree:
          TableScan
            alias: t2
            Statistics: Num rows: 3536 Data size: 1131711 Basic stats: COMPLETE Column stats: NONE
            Map Join Operator
              condition map:
                   Inner Join 0 to 1
              condition expressions:
                0 {_col0}
                1 {ctinyint} {csmallint} {cint} {cbigint} {cfloat} {cdouble} {cstring1} {cstring2} {ctimestamp1} {ctimestamp2} {cboolean1} {cboolean2}
              keys:
                0 _col0 (type: int)
                1 cint (type: int)
              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
              Statistics: Num rows: 3889 Data size: 1244882 Basic stats: COMPLETE Column stats: NONE
              Filter Operator
                predicate: (_col0 = _col3) (type: boolean)
                Statistics: Num rows: 1944 Data size: 622280 Basic stats: COMPLETE Column stats: NONE
                Select Operator
                  expressions: _col0 (type: int), _col1 (type: tinyint), _col2 (type: smallint), _col3 (type: int), _col4 (type: bigint), _col5 (type: float), _col6 (type: double), _col7 (type: string), _col8 (type: string), _col\
9 (type: timestamp), _col10 (type: timestamp), _col11 (type: boolean), _col12 (type: boolean)
                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                  Statistics: Num rows: 1944 Data size: 622280 Basic stats: COMPLETE Column stats: NONE
                  Reduce Output Operator
                    key expressions: _col9 (type: timestamp)
                    sort order: +
                    Statistics: Num rows: 1944 Data size: 622280 Basic stats: COMPLETE Column stats: NONE
                    value expressions: _col0 (type: int), _col1 (type: tinyint), _col2 (type: smallint), _col3 (type: int), _col4 (type: bigint), _col5 (type: float), _col6 (type: double), _col7 (type: string), _col8 (type: strin\
g), _col9 (type: timestamp), _col10 (type: timestamp), _col11 (type: boolean), _col12 (type: boolean)
      Local Work:
        Map Reduce Local Work
      Execution mode: vectorized
      Reduce Operator Tree:
        Extract
          Statistics: Num rows: 1944 Data size: 622280 Basic stats: COMPLETE Column stats: NONE
          Limit
            Number of rows: 100
            Statistics: Num rows: 100 Data size: 32000 Basic stats: COMPLETE Column stats: NONE
            File Output Operator
              compressed: false
              Statistics: Num rows: 100 Data size: 32000 Basic stats: COMPLETE Column stats: NONE
              table:
                  input format: org.apache.hadoop.mapred.TextInputFormat
                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe

  Stage: Stage-0
    Fetch Operator
      limit: 100
where as with the current code, vectorization fails to take place because of the following exception
14/03/12 14:43:19 DEBUG vector.VectorizationContext: No vector udf found for GenericUDFOPEqual, descriptor: Argument Count = 2, mode = FILTER, Argument Types = {STRING,LONG}, Input Expression Types = {COLUMN,COLUMN}
14/03/12 14:43:19 DEBUG physical.Vectorizer: Failed to vectorize
org.apache.hadoop.hive.ql.metadata.HiveException: Udf: GenericUDFOPEqual, is not supported
	at org.apache.hadoop.hive.ql.exec.vector.VectorizationContext.getGenericUdfVectorExpression(VectorizationContext.java:854)
	at org.apache.hadoop.hive.ql.exec.vector.VectorizationContext.getVectorExpression(VectorizationContext.java:300)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.validateExprNodeDesc(Vectorizer.java:682)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.validateFilterOperator(Vectorizer.java:606)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.validateOperator(Vectorizer.java:537)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer$ValidationNodeProcessor.process(Vectorizer.java:367)
	at org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher.dispatch(DefaultRuleDispatcher.java:90)
	at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.dispatchAndReturn(DefaultGraphWalker.java:94)
	at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.dispatch(DefaultGraphWalker.java:78)
	at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.walk(DefaultGraphWalker.java:132)
	at org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.startWalking(DefaultGraphWalker.java:109)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer$VectorizationDispatcher.validateMapWork(Vectorizer.java:314)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer$VectorizationDispatcher.convertMapWork(Vectorizer.java:283)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer$VectorizationDispatcher.dispatch(Vectorizer.java:270)
	at org.apache.hadoop.hive.ql.lib.TaskGraphWalker.dispatch(TaskGraphWalker.java:111)
	at org.apache.hadoop.hive.ql.lib.TaskGraphWalker.walk(TaskGraphWalker.java:194)
	at org.apache.hadoop.hive.ql.lib.TaskGraphWalker.startWalking(TaskGraphWalker.java:139)
	at org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.resolve(Vectorizer.java:519)
	at org.apache.hadoop.hive.ql.optimizer.physical.PhysicalOptimizer.optimize(PhysicalOptimizer.java:100)
	at org.apache.hadoop.hive.ql.parse.MapReduceCompiler.optimizeTaskPlan(MapReduceCompiler.java:290)
	at org.apache.hadoop.hive.ql.parse.TaskCompiler.compile(TaskCompiler.java:216)
	at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:9286)
	at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:327)
	at org.apache.hadoop.hive.ql.parse.ExplainSemanticAnalyzer.analyzeInternal(ExplainSemanticAnalyzer.java:64)
	at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:327)
	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:398)
	at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:294)
	at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:948)
	at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:996)
	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:884)
	at org.apache.hadoop.hive.ql.Driver.run(Driver.java:874)
	at org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:268)
	at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:220)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:424)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:359)
	at org.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:457)
	at org.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:467)
	at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:125)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:424)
	at org.apache.hadoop.hive.cli.CliDriver.executeDriver(CliDriver.java:793)
	at org.apache.hadoop.hive.cli.CliDriver.run(CliDriver.java:687)
	at org.apache.hadoop.hive.cli.CliDriver.main(CliDriver.java:626)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
	at java.lang.reflect.Method.invoke(Method.java:597)
	at org.apache.hadoop.util.RunJar.main(RunJar.java:160)




--
This message was sent by Atlassian JIRA
(v6.2#6252)