You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "LuGuangMing (Jira)" <ji...@apache.org> on 2020/09/01 03:14:00 UTC

[jira] [Created] (HIVE-24100) Syntax compile failure occurs when INSERT table column Order by is greater than 2 columns when CBO is false

LuGuangMing created HIVE-24100:
----------------------------------

             Summary: Syntax compile failure occurs when INSERT table column Order by is greater than 2 columns when CBO is false
                 Key: HIVE-24100
                 URL: https://issues.apache.org/jira/browse/HIVE-24100
             Project: Hive
          Issue Type: Bug
          Components: CBO, Hive
    Affects Versions: 3.1.2, 3.1.0
            Reporter: LuGuangMing


Executing the following SQL will fail to compile
{code:java}
set hive.cbo.enable=false;create table tab_1
(
item_id string, 
stru_area_id string
)partitioned by ( PT_DT string) stored as orc;create table tab_2
(
CREATE_ORG_ID string,
PROMOTION_ID  string,
PROMOTION_STATUS string
) partitioned by (pt_dt string) stored as orc;create table tab_3
(
STRU_ID string,
SUP_STRU string
) partitioned by(pt_dt string) stored as orc;
explain
insert into table tab_1 partition(PT_DT = '2020-08-22')
(item_id , stru_area_id)
select '123' ITEM_ID , T.STRU_ID STRU_AREA_ID 
from ( 
  select 
  T0.STRU_ID STRU_ID ,T0.STRU_ID STRU_ID_BRANCH 
  from  tab_3 T0 
) T
inner join ( 
  select 
  TT.CREATE_ORG_ID
  from  tab_2 TT 
) TIV
on (T.STRU_ID_BRANCH = TIV.CREATE_ORG_ID) 
group by T.STRU_ID
order by 1,2;
{code}
{code:java}
org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: SemanticException [Error 10004]: Line 5:28 Invalid table alias or column reference 'T': (possible column names are: _col0, _col1)
 at org.apache.hive.service.cli.operation.Operation.toSQLException(Operation.java:341) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:215) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.operation.SQLOperation.runInternal(SQLOperation.java:316) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.operation.Operation.run(Operation.java:253) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementInternal(HiveSessionImpl.java:684) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.session.HiveSessionImpl.executeStatementAsync(HiveSessionImpl.java:670) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.CLIService.executeStatementAsync(CLIService.java:342) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.thrift.ThriftCLIService.executeNewStatement(ThriftCLIService.java:1144) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.thrift.ThriftCLIService.ExecuteStatement(ThriftCLIService.java:1280) ~[hive-service-3.1.0.jar:3.1.0]
 at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1557) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hive.service.rpc.thrift.TCLIService$Processor$ExecuteStatement.getResult(TCLIService.java:1542) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:648) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286) ~[hive-exec-3.1.0.jar:3.1.0]
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_201]
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_201]
 at java.lang.Thread.run(Thread.java:748) [?:1.8.0_201]
Caused by: org.apache.hadoop.hive.ql.parse.SemanticException: Line 5:28 Invalid table alias or column reference 'T': (possible column names are: _col0, _col1)
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genAllExprNodeDesc(SemanticAnalyzer.java:12689) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genExprNodeDesc(SemanticAnalyzer.java:12629) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genExprNodeDesc(SemanticAnalyzer.java:12597) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genExprNodeDesc(SemanticAnalyzer.java:12575) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genReduceSinkPlan(SemanticAnalyzer.java:8482) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPostGroupByBodyPlan(SemanticAnalyzer.java:10616) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genBodyPlan(SemanticAnalyzer.java:10515) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:11434) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:11304) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genOPTree(SemanticAnalyzer.java:12090) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:12180) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:11692) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:281) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.ExplainSemanticAnalyzer.analyzeInternal(ExplainSemanticAnalyzer.java:164) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:281) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:712) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:2117) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:2064) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.Driver.compileAndRespond(Driver.java:2059) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hadoop.hive.ql.reexec.ReExecDriver.compileAndRespond(ReExecDriver.java:137) ~[hive-exec-3.1.0.jar:3.1.0]
 at org.apache.hive.service.cli.operation.SQLOperation.prepare(SQLOperation.java:204) ~[hive-service-3.1.0.jar:3.1.0]
 ... 16 more
{code}
 

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)