You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "Jason Dere (JIRA)" <ji...@apache.org> on 2015/11/20 04:31:10 UTC

[jira] [Updated] (HIVE-12476) Metastore NPE on Oracle with Direct SQL

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

Jason Dere updated HIVE-12476:
------------------------------
    Description: 
Stack trace looks very similar to HIVE-8485. I believe the metastore's Direct SQL mode requires additional fixes similar to HIVE-8485, around the Partition/StorageDescriptorSerDe parameters.

{noformat}
2015-11-19 18:08:33,841 ERROR [pool-5-thread-2]: server.TThreadPoolServer (TThreadPoolServer.java:run(296)) - Error occurred during processing of message.
java.lang.NullPointerException
        at org.apache.thrift.protocol.TBinaryProtocol.writeString(TBinaryProtocol.java:200)
        at org.apache.hadoop.hive.metastore.api.SerDeInfo$SerDeInfoStandardScheme.write(SerDeInfo.java:579)
        at org.apache.hadoop.hive.metastore.api.SerDeInfo$SerDeInfoStandardScheme.write(SerDeInfo.java:501)
        at org.apache.hadoop.hive.metastore.api.SerDeInfo.write(SerDeInfo.java:439)
        at org.apache.hadoop.hive.metastore.api.StorageDescriptor$StorageDescriptorStandardScheme.write(StorageDescriptor.java:1490)
        at org.apache.hadoop.hive.metastore.api.StorageDescriptor$StorageDescriptorStandardScheme.write(StorageDescriptor.java:1288)
        at org.apache.hadoop.hive.metastore.api.StorageDescriptor.write(StorageDescriptor.java:1154)
        at org.apache.hadoop.hive.metastore.api.Partition$PartitionStandardScheme.write(Partition.java:1072)
        at org.apache.hadoop.hive.metastore.api.Partition$PartitionStandardScheme.write(Partition.java:929)
        at org.apache.hadoop.hive.metastore.api.Partition.write(Partition.java:825)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_result$get_partitions_resultStandardScheme.write(ThriftHiveMetastore.java:64470)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_result$get_partitions_resultStandardScheme.write(ThriftHiveMetastore.java:64402)
        at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_result.write(ThriftHiveMetastore.java:64340)
        at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:53)
        at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:681)
        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:676)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
        at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:676)
        at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:285)        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
{noformat}

  was:Stack trace looks very similar to HIVE-8485. I believe the metastore's Direct SQL mode requires additional fixes similar to HIVE-8485, around the Partition/StorageDescriptorSerDe parameters.


> Metastore NPE on Oracle with Direct SQL
> ---------------------------------------
>
>                 Key: HIVE-12476
>                 URL: https://issues.apache.org/jira/browse/HIVE-12476
>             Project: Hive
>          Issue Type: Bug
>          Components: Metastore
>            Reporter: Jason Dere
>            Assignee: Jason Dere
>
> Stack trace looks very similar to HIVE-8485. I believe the metastore's Direct SQL mode requires additional fixes similar to HIVE-8485, around the Partition/StorageDescriptorSerDe parameters.
> {noformat}
> 2015-11-19 18:08:33,841 ERROR [pool-5-thread-2]: server.TThreadPoolServer (TThreadPoolServer.java:run(296)) - Error occurred during processing of message.
> java.lang.NullPointerException
>         at org.apache.thrift.protocol.TBinaryProtocol.writeString(TBinaryProtocol.java:200)
>         at org.apache.hadoop.hive.metastore.api.SerDeInfo$SerDeInfoStandardScheme.write(SerDeInfo.java:579)
>         at org.apache.hadoop.hive.metastore.api.SerDeInfo$SerDeInfoStandardScheme.write(SerDeInfo.java:501)
>         at org.apache.hadoop.hive.metastore.api.SerDeInfo.write(SerDeInfo.java:439)
>         at org.apache.hadoop.hive.metastore.api.StorageDescriptor$StorageDescriptorStandardScheme.write(StorageDescriptor.java:1490)
>         at org.apache.hadoop.hive.metastore.api.StorageDescriptor$StorageDescriptorStandardScheme.write(StorageDescriptor.java:1288)
>         at org.apache.hadoop.hive.metastore.api.StorageDescriptor.write(StorageDescriptor.java:1154)
>         at org.apache.hadoop.hive.metastore.api.Partition$PartitionStandardScheme.write(Partition.java:1072)
>         at org.apache.hadoop.hive.metastore.api.Partition$PartitionStandardScheme.write(Partition.java:929)
>         at org.apache.hadoop.hive.metastore.api.Partition.write(Partition.java:825)
>         at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_result$get_partitions_resultStandardScheme.write(ThriftHiveMetastore.java:64470)
>         at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_result$get_partitions_resultStandardScheme.write(ThriftHiveMetastore.java:64402)
>         at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_result.write(ThriftHiveMetastore.java:64340)
>         at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:53)
>         at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
>         at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:681)
>         at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:676)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:422)
>         at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
>         at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:676)
>         at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:285)        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}



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