Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-4211

Common column and partition column are defined the same type and union them, it will hints Schema of both sides of union should match.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Patch Available
    • Major
    • Resolution: Unresolved
    • 0.9.0, 0.11.0
    • None
    • SQL

    Description

      create table UnionBoolA (id boolean, no boolean) row format delimited fields terminated by ' ';
      load data local inpath '/opt/files/unionboola.txt' into table UnionBoolA;
      create table UnionPartionBool (id int) partitioned by (no boolean) row format delimited fields terminated by ' ';
      load data local inpath '/opt/files/unionpartint.txt' into table UnionPartionBool partition(no=true);

      unionboola.txt:
      true true
      false true
      true true
      false true

      unionpartint.txt:
      111
      444
      1122
      44

      when I execute
      "select * from( select no from UnionBoolA union all select no from UnionPartionBool) unionResult", it is failed. The exception info is as follows:
      FAILED: Error in semantic analysis: 1:66 Schema of both sides of union should match: Column no is of type boolean on first table and type string on second table. Error encountered near token 'UnionPartionBool'
      org.apache.hadoop.hive.ql.parse.SemanticException: 1:66 Schema of both sides of union should match: Column no is of type boolean on first table and type string on second table. Error encountered near token 'UnionPartionBool'
      at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genUnionPlan(SemanticAnalyzer.java:6295)
      at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:6733)
      at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genPlan(SemanticAnalyzer.java:6748)
      at org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.analyzeInternal(SemanticAnalyzer.java:7556)
      at org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.analyze(BaseSemanticAnalyzer.java:244)
      at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:621)
      at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:525)
      at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1153)
      at org.apache.hadoop.hive.service.HiveServer$HiveServerHandler.execute(HiveServer.java:226)
      at org.apache.hadoop.hive.service.ThriftHive$Processor$execute.getResult(ThriftHive.java:630)
      at org.apache.hadoop.hive.service.ThriftHive$Processor$execute.getResult(ThriftHive.java:618)
      at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:32)
      at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:34)
      at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge20S$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge20S.java:535)
      at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge20S$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge20S.java:532)
      at java.security.AccessController.doPrivileged(Native Method)
      at javax.security.auth.Subject.doAs(Subject.java:396)
      at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1232)
      at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge20S$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge20S.java:532)
      at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:176)
      at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
      at java.lang.Thread.run(Thread.java:662)

      So I execute "explain select no from UnionPartionBool" to see the partition column, and find the partition column type is string.
      And all the partition column type is changed to be TypeInfoFactory.stringTypeInfo. It is in org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.genTablePlan(). And it is todo task. Now I modify it to be TypeInfoFactory.getPrimitiveTypeInfo(part_col.getType()).It can fix this bug.
      And you can see what I modified in the patch.

      Attachments

        1. PartitionColumnTypInfo.patch
          0.9 kB
          Daisy.Yuan
        2. HIVE-4211.patch
          0.6 kB
          Daisy.Yuan

        Activity

          People

            Unassigned Unassigned
            daisy_yu Daisy.Yuan
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated: