[ 
https://issues.apache.org/jira/browse/HIVE-9185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Slava Markeyev updated HIVE-9185:
---------------------------------
    Summary: NPE in ColumnPruner pruneReduceSinkOperator  (was: NPE in 
ColumnPruner pruneReduceSinkOperator when )

> NPE in ColumnPruner pruneReduceSinkOperator
> -------------------------------------------
>
>                 Key: HIVE-9185
>                 URL: https://issues.apache.org/jira/browse/HIVE-9185
>             Project: Hive
>          Issue Type: Bug
>          Components: Query Processor
>    Affects Versions: 0.13.1
>            Reporter: Slava Markeyev
>
> When nm is null reverseLookup(outputCol) is called which also returns null 
> for the outputColumn lookup.
> Query:
> {code}
> SELECT session_aggregate.app_frequency AS number_sessions, 
> COUNT(session_aggregate.s) AS users
> FROM (
>     SELECT session_info.s, MAX(session_info.session_number) AS app_frequency
>     FROM (
>         SELECT kt_session(calc_session.s, calc_session.evt_lst,2) AS (s, 
> session_number, session_length)
>         FROM (
>             SELECT session_set.s, collect_set(session_set.timestamps) evt_lst
>             FROM (
>                 SELECT s, evt.utc_timestamp AS timestamps
>                 FROM event evt
>                 WHERE month = 201412
>             ) session_set
>         GROUP BY session_set.s
>         ) calc_session
>     ORDER BY s,session_number DESC
>     )session_info
> GROUP BY session_info.s
> )session_aggregate
> GROUP BY session_aggregate.app_frequency
> {code}
> Trace log on explain of query:
> {code}
> 14/12/21 01:10:41 INFO parse.ParseDriver: Parsing command: explain SELECT 
> session_aggregate.app_frequency AS number_sessions, 
> COUNT(session_aggregate.s) AS users
> FROM (
>     SELECT session_info.s, MAX(session_info.session_number) AS app_frequency
>     FROM (
>         SELECT kt_session(calc_session.s, calc_session.evt_lst,2) AS (s, 
> session_number, session_length)
>         FROM (
>             SELECT session_set.s, collect_set(session_set.timestamps) evt_lst
>             FROM (
>                 SELECT s, evt.utc_timestamp AS timestamps
>                 FROM event evt
>                 WHERE month = 201412
>             ) session_set
>         GROUP BY session_set.s
>         ) calc_session
>     ORDER BY s,session_number DESC
>     )session_info
> GROUP BY session_info.s
> )session_aggregate
> GROUP BY session_aggregate.app_frequency
> 14/12/21 01:10:41 INFO parse.ParseDriver: Parse Completed
> 14/12/21 01:10:41 INFO log.PerfLogger: </PERFLOG method=parse 
> start=1419124241784 end=1419124241810 duration=26 
> from=org.apache.hadoop.hive.ql.Driver>
> 14/12/21 01:10:41 INFO log.PerfLogger: <PERFLOG method=semanticAnalyze 
> from=org.apache.hadoop.hive.ql.Driver>
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Starting Semantic Analysis
> 14/12/21 01:10:41 DEBUG exec.FunctionRegistry: Looking up GenericUDAF: 
> collect_set
> 14/12/21 01:10:41 DEBUG exec.FunctionRegistry: Looking up GenericUDAF: 
> kt_session
> 14/12/21 01:10:41 DEBUG exec.FunctionRegistry: Looking up GenericUDAF: MAX
> 14/12/21 01:10:41 DEBUG exec.FunctionRegistry: Looking up GenericUDAF: COUNT
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Completed phase 1 of Semantic 
> Analysis
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for source tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for subqueries
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for source tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for subqueries
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for source tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for subqueries
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for source tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for subqueries
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for source tables
> 14/12/21 01:10:41 WARN conf.Configuration: 
> org.apache.hadoop.hive.conf.LoopingByteArrayInputStream@52b84700:an attempt 
> to override final parameter: mapred.reduce.tasks.speculative.execution;  
> Ignoring.
> 14/12/21 01:10:41 DEBUG hive.log: DDL: struct event { i64 utc_timestamp, 
> string type, string s, string n, i32 v, i32 l, string st1, string st2, string 
> st3, string json_data, string ts}
> 14/12/21 01:10:41 DEBUG hive.log: DDL: struct event { i64 utc_timestamp, 
> string type, string s, string n, i32 v, i32 l, string st1, string st2, string 
> st3, string json_data, string ts}
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for subqueries
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for destination 
> tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for destination 
> tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for destination 
> tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for destination 
> tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Get metadata for destination 
> tables
> 14/12/21 01:10:41 INFO parse.SemanticAnalyzer: Completed getting MetaData in 
> Semantic Analysis
> 14/12/21 01:10:42 DEBUG exec.FunctionRegistry: Looking up GenericUDAF: 
> collect_set
> 14/12/21 01:10:42 DEBUG exec.FunctionRegistry: Looking up GenericUDAF: MAX
> 14/12/21 01:10:42 DEBUG exec.FunctionRegistry: Looking up GenericUDAF: COUNT
> 14/12/21 01:10:42 DEBUG lazy.LazySimpleSerDe: 
> org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe initialized with: 
> columnNames=[_col0, _col1] columnTypes=[int, bigint] separator=[[B@45045faa] 
> nullstring=\N lastColumnTakesRest=false
> 14/12/21 01:10:42 DEBUG lazy.LazySimpleSerDe: 
> org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe initialized with: 
> columnNames=[_col0, _col1] columnTypes=[int, bigint] separator=[[B@9fa27e9] 
> nullstring=\N lastColumnTakesRest=false
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for FS(22)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(21)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for GBY(20)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for RS(19)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for GBY(18)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(17)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(16)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for GBY(15)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for RS(14)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for GBY(13)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(12)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for EX(11)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for RS(10)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for UDTF(9)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(8)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(7)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for GBY(6)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for RS(5)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for GBY(4)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(3)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for SEL(2)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for FIL(1)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Pushdown Predicates of FIL For 
> Alias : evt
> 14/12/21 01:10:42 INFO ppd.OpProcFactory:     (month = 201412)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Processing for TS(0)
> 14/12/21 01:10:42 INFO ppd.OpProcFactory: Pushdown Predicates of TS For Alias 
> : evt
> 14/12/21 01:10:42 INFO ppd.OpProcFactory:     (month = 201412)
> 14/12/21 01:10:42 TRACE ppr.PartitionPruner: Started pruning partiton
> 14/12/21 01:10:42 TRACE ppr.PartitionPruner: dbname = default
> 14/12/21 01:10:42 TRACE ppr.PartitionPruner: tabname = event
> 14/12/21 01:10:42 TRACE ppr.PartitionPruner: prune Expression = 
> GenericUDFOPEqual(Column[month], Const int 201412)
> 14/12/21 01:10:42 DEBUG ppr.PartitionPruner: tabname = event is partitioned
> 14/12/21 01:10:42 DEBUG ppr.PartitionPruner: Filter w/ compacting: (month = 
> 201412); filter w/o compacting: (month = 201412)
> 14/12/21 01:10:42 INFO log.PerfLogger: <PERFLOG method=partition-retrieving 
> from=org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner>
> 14/12/21 01:10:42 INFO log.PerfLogger: </PERFLOG method=partition-retrieving 
> start=1419124242223 end=1419124242320 duration=97 
> from=org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner>
> 14/12/21 01:10:42 DEBUG hive.log: DDL: struct event { i64 utc_timestamp, 
> string type, string s, string n, i32 v, i32 l, string st1, string st2, string 
> st3, string json_data, string ts}
> 14/12/21 01:10:42 INFO optimizer.ColumnPrunerProcFactory: RS 19 
> oldColExprMap: {KEY._col0=Column[_col0], VALUE._col0=Column[_col1]}
> 14/12/21 01:10:42 INFO optimizer.ColumnPrunerProcFactory: RS 19 
> newColExprMap: {KEY._col0=Column[_col0], VALUE._col0=Column[_col1]}
> 14/12/21 01:10:42 INFO optimizer.ColumnPrunerProcFactory: RS 14 
> oldColExprMap: {KEY._col0=Column[_col0], VALUE._col0=Column[_col1]}
> 14/12/21 01:10:42 INFO optimizer.ColumnPrunerProcFactory: RS 14 
> newColExprMap: {KEY._col0=Column[_col0], VALUE._col0=Column[_col1]}
> 14/12/21 01:10:42 INFO optimizer.ColumnPrunerProcFactory: RS 10 
> oldColExprMap: {_col2=Column[sessionlength], _col1=Column[sessionid], 
> _col0=Column[s]}
> FAILED: NullPointerException null
> 14/12/21 01:10:42 ERROR ql.Driver: FAILED: NullPointerException null
> java.lang.NullPointerException
>       at 
> org.apache.hadoop.hive.ql.optimizer.ColumnPrunerProcFactory.pruneReduceSinkOperator(ColumnPrunerProcFactory.java:715)
>       at 
> org.apache.hadoop.hive.ql.optimizer.ColumnPrunerProcFactory.access$200(ColumnPrunerProcFactory.java:78)
>       at 
> org.apache.hadoop.hive.ql.optimizer.ColumnPrunerProcFactory$ColumnPrunerReduceSinkProc.process(ColumnPrunerProcFactory.java:427)
>       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.optimizer.ColumnPruner$ColumnPrunerWalker.walk(ColumnPruner.java:166)
>       at 
> org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.startWalking(DefaultGraphWalker.java:109)
>       at 
> org.apache.hadoop.hive.ql.optimizer.ColumnPruner.transform(ColumnPruner.java:129)
>       at 
> org.apache.hadoop.hive.ql.optimizer.Optimizer.optimize(Optimizer.java:146)
>       at 
> org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:9268)
>       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:422)
>       at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:322)
>       at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:975)
>       at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1040)
>       at org.apache.hadoop.hive.ql.Driver.run(Driver.java:911)
>       at org.apache.hadoop.hive.ql.Driver.run(Driver.java:901)
>       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:423)
>       at 
> org.apache.hadoop.hive.cli.CliDriver.executeDriver(CliDriver.java:792)
>       at org.apache.hadoop.hive.cli.CliDriver.run(CliDriver.java:686)
>       at org.apache.hadoop.hive.cli.CliDriver.main(CliDriver.java:625)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:606)
>       at org.apache.hadoop.util.RunJar.main(RunJar.java:212)
> {code}



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

Reply via email to