Uploaded image for project: 'Phoenix'
  1. Phoenix
  2. PHOENIX-4222

PhoenixStorageHandler for Hive is not working

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 4.11.0
    • None
    • None

    Description

      ERROR CliDriver: Failed with exception java.io.IOException:java.lang.RuntimeException: org.apache.phoenix.schema.ColumnNotFoundException: ERROR 504 (42703): Undefined column. columnName=PHOENIX.MOVIES.movie_id
      java.io.IOException: java.lang.RuntimeException: org.apache.phoenix.schema.ColumnNotFoundException: ERROR 504 (42703): Undefined column. columnName=PHOENIX.MOVIES.movie_id
      at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:508)
      at org.apache.hadoop.hive.ql.exec.FetchOperator.pushRow(FetchOperator.java:415)
      at org.apache.hadoop.hive.ql.exec.FetchTask.fetch(FetchTask.java:138)
      at org.apache.hadoop.hive.ql.Driver.getResults(Driver.java:1986)
      at org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:240)
      at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:172)
      at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:383)
      at org.apache.hadoop.hive.cli.CliDriver.executeDriver(CliDriver.java:775)
      at org.apache.hadoop.hive.cli.CliDriver.run(CliDriver.java:693)
      at org.apache.hadoop.hive.cli.CliDriver.main(CliDriver.java:628)
      Caused by: java.lang.RuntimeException: org.apache.phoenix.schema.ColumnNotFoundException: ERROR 504 (42703): Undefined column. columnName=PHOENIX.MOVIES.movie_id
      at org.apache.phoenix.hive.mapreduce.PhoenixInputFormat.getQueryPlan(PhoenixInputFormat.java:266)
      at org.apache.phoenix.hive.mapreduce.PhoenixInputFormat.getSplits(PhoenixInputFormat.java:131)
      at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextSplits(FetchOperator.java:363)
      at org.apache.hadoop.hive.ql.exec.FetchOperator.getRecordReader(FetchOperator.java:295)
      at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:446)
      ... 9 more
      Caused by: org.apache.phoenix.schema.ColumnNotFoundException: ERROR 504 (42703): Undefined column. columnName=PHOENIX.MOVIES.movie_id
      at org.apache.phoenix.schema.PTableImpl.getColumnForColumnName(PTableImpl.java:818)
      at org.apache.phoenix.compile.FromCompiler$SingleTableColumnResolver.resolveColumn(FromCompiler.java:478)
      at org.apache.phoenix.compile.TupleProjectionCompiler$ColumnRefVisitor.visit(TupleProjectionCompiler.java:208)
      at org.apache.phoenix.compile.TupleProjectionCompiler$ColumnRefVisitor.visit(TupleProjectionCompiler.java:1)
      at org.apache.phoenix.parse.ColumnParseNode.accept(ColumnParseNode.java:56)
      at org.apache.phoenix.compile.TupleProjectionCompiler.createProjectedTable(TupleProjectionCompiler.java:109)
      at org.apache.phoenix.compile.QueryCompiler.compileSingleFlatQuery(QueryCompiler.java:528)
      at org.apache.phoenix.compile.QueryCompiler.compileSingleQuery(QueryCompiler.java:507)
      at org.apache.phoenix.compile.QueryCompiler.compileSelect(QueryCompiler.java:202)
      at org.apache.phoenix.compile.QueryCompiler.compile(QueryCompiler.java:157)
      at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:475)
      at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:1)
      at org.apache.phoenix.jdbc.PhoenixStatement.compileQuery(PhoenixStatement.java:1648)
      at org.apache.phoenix.jdbc.PhoenixStatement.compileQuery(PhoenixStatement.java:1641)
      at org.apache.phoenix.jdbc.PhoenixStatement.optimizeQuery(PhoenixStatement.java:1635)
      at org.apache.phoenix.hive.mapreduce.PhoenixInputFormat.getQueryPlan(PhoenixInputFormat.java:260)
      ... 13 more

      : jdbc:phoenix:juke-cdh2> !columns phoenix.movies
      +------------+--------------+-------------+--------------+------------+------------+--------------+----------------+-----------------+-----------------+-----------+----------+-------------+
      | TABLE_CAT  | TABLE_SCHEM  | TABLE_NAME  | COLUMN_NAME  | DATA_TYPE  | TYPE_NAME  | COLUMN_SIZE  | BUFFER_LENGTH  | DECIMAL_DIGITS  | NUM_PREC_RADIX  | NULLABLE  | REMARKS  | COLUMN_DEF  |
      +------------+--------------+-------------+--------------+------------+------------+--------------+----------------+-----------------+-----------------+-----------+----------+-------------+
      |            | PHOENIX      | MOVIES      | MOVIE_ID     | 4          | INTEGER    | null         | null           | null            | null            | 0         |          |             |
      |            | PHOENIX      | MOVIES      | TITLE        | 12         | VARCHAR    | null         | null           | null            | null            | 1         |          |             |
      |            | PHOENIX      | MOVIES      | GENRES       | 12         | VARCHAR    | null         | null           | null            | null            | 1         |          |             |
      +------------+--------------+-------------+--------------+------------+------------+--------------+----------------+-----------------+-----------------+-----------+----------+-------------+
      
      hive> show create table movies;
      CREATE EXTERNAL TABLE `movies`(
        `movie_id` int COMMENT 'from deserializer', 
        `title` string COMMENT 'from deserializer', 
        `genres` string COMMENT 'from deserializer')
      ROW FORMAT SERDE 
        'org.apache.phoenix.hive.PhoenixSerDe' 
      STORED BY 
        'org.apache.phoenix.hive.PhoenixStorageHandler' 
      WITH SERDEPROPERTIES ( 
        'serialization.format'='1')
      LOCATION
        'hdfs://nameservice1/user/hive/warehouse/movies'
      TBLPROPERTIES (
        'COLUMN_STATS_ACCURATE'='false', 
        'numFiles'='0', 
        'numRows'='-1', 
        'phoenix.table.name'='phoenix.movies', 
        'phoenix.zookeeper.quorum'='xxx', 
        'rawDataSize'='-1', 
        'totalSize'='0', 
        'transient_lastDdlTime'='1503021635')
      

      I ran the query "select * from movies" and
      The query created by the phoenix handler is:
      select /*+ NO_CACHE */ "movie_id","title","genres" from phoenix.movies
      But do not enter double quotes like this.
      I looked up where to put double quotes.


      Why did you change it to double quotes?

      Attachments

        Issue Links

          Activity

            People

              sergey.soldatov Sergey Soldatov
              mini666 JeongMin Ju
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated: