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

Harish Jaiprakash updated HIVE-19646:
-------------------------------------
    Status: Patch Available  (was: Open)

Create patch with following changes:
 * Always get filesystem object from the path, instead of storing it.
 * Retry logic when logging fails.
 * Moved the code copied from tez to the tez folders, this is to make sure we 
can remove this code in a backwards compatible way, when hive moves to next 
release tez version. SequenceFile writes the key and value class in the file 
and prevents being read if the class package name is different.

> Filesystem closed error in HiveProtoLoggingHook
> -----------------------------------------------
>
>                 Key: HIVE-19646
>                 URL: https://issues.apache.org/jira/browse/HIVE-19646
>             Project: Hive
>          Issue Type: Bug
>            Reporter: Harish Jaiprakash
>            Assignee: Harish Jaiprakash
>            Priority: Major
>         Attachments: HIVE-19646.01.patch
>
>
> Exception in proto logging hook on secure cluster.
> {code}
> 2018-05-18T04:48:01,136 ERROR [Hive Hook Proto Log Writer 0]: 
> hooks.HiveProtoLoggingHook (:()) - Error writing proto message for query 
> hive_20180518043717_ca3ab4df-6cab-4920-aa44-2340ae246ad2, eventType: 
> QUERY_SUBMITTED:
> java.io.IOException: Filesystem closed
>  at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:472) 
> ~[hadoop-hdfs-client-3.0.0.3.0.0.0-1298.jar:?]
>  at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1652) 
> ~[hadoop-hdfs-client-3.0.0.3.0.0.0-1298.jar:?]
>  at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1569)
>  ~[hadoop-hdfs-client-3.0.0.3.0.0.0-1298.jar:?]
>  at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1566)
>  ~[hadoop-hdfs-client-3.0.0.3.0.0.0-1298.jar:?]
>  at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>  ~[hadoop-common-3.0.0.3.0.0.0-1298.jar:?]
>  at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1581)
>  ~[hadoop-hdfs-client-3.0.0.3.0.0.0-1298.jar:?]
>  at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1734) 
> ~[hadoop-common-3.0.0.3.0.0.0-1298.jar:?]
>  at 
> org.apache.hadoop.hive.ql.hooks.DatePartitionedLogger.getPathForDate(DatePartitionedLogger.java:89)
>  ~[hive-exec-3.0.0.3.0.0.0-1298.jar:3.0.0.3.0.0.0-1298]
>  at 
> org.apache.hadoop.hive.ql.hooks.DatePartitionedLogger.getWriter(DatePartitionedLogger.java:73)
>  ~[hive-exec-3.0.0.3.0.0.0-1298.jar:3.0.0.3.0.0.0-1298]
>  at 
> org.apache.hadoop.hive.ql.hooks.HiveProtoLoggingHook$EventLogger.writeEvent(HiveProtoLoggingHook.java:283)
>  ~[hive-exec-3.0.0.3.0.0.0-1298.jar:3.0.0.3.0.0.0-1298]
>  at 
> org.apache.hadoop.hive.ql.hooks.HiveProtoLoggingHook$EventLogger.lambda$generateEvent$1(HiveProtoLoggingHook.java:274)
>  ~[hive-exec-3.0.0.3.0.0.0-1298.jar:3.0.0.3.0.0.0-1298]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  ~[?:1.8.0_161]
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  ~[?:1.8.0_161]
>  at java.lang.Thread.run(Thread.java:748) [?:1.8.0_161]
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to