You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "Wei-Chiu Chuang (Jira)" <ji...@apache.org> on 2022/01/11 06:53:00 UTC

[jira] [Commented] (HIVE-25806) Possible leak in LlapCacheAwareFs - Parquet, LLAP IO

    [ https://issues.apache.org/jira/browse/HIVE-25806?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17472495#comment-17472495 ] 

Wei-Chiu Chuang commented on HIVE-25806:
----------------------------------------

For future reference, this bug was detected using the tool http://file-leak-detector.kohsuke.org/
Kudos to Laszo for fixing this tricky bug!

> Possible leak in LlapCacheAwareFs - Parquet, LLAP IO
> ----------------------------------------------------
>
>                 Key: HIVE-25806
>                 URL: https://issues.apache.org/jira/browse/HIVE-25806
>             Project: Hive
>          Issue Type: Bug
>            Reporter: László Bodor
>            Assignee: László Bodor
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 4.0.0
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> there is an inputstream there which is never closed:
> https://github.com/apache/hive/blob/9f9844dbc881e2a9267c259b8c04e7787f7fadc4/ql/src/java/org/apache/hadoop/hive/llap/LlapCacheAwareFs.java#L243
> my understanding is that in an InputStream chain, every InputStream is responsible for closing its enclosed InputStream, here the chain is like:
> DelegatingSeekableInputStream -> io.DataInputStream -> LlapCacheAwareFs$CacheAwareInputStream -> io.DataInputStream -> crypto.CryptoInputStream -> hdfs.DFSInputStream
> {code}
> 	at sun.nio.ch.SocketChannelImpl.<init>(SocketChannelImpl.java:106)
> 	at sun.nio.ch.SelectorProviderImpl.openSocketChannel(SelectorProviderImpl.java:60)
> 	at java.nio.channels.SocketChannel.open(SocketChannel.java:145)
> 	at org.apache.hadoop.net.StandardSocketFactory.createSocket(StandardSocketFactory.java:62)
> 	at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2933)
> 	at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:821)
> 	at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:746)
> 	at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:379)
> 	at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:644)
> 	at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:575)
> 	at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:757)
> 	at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:836)
> 	at org.apache.hadoop.crypto.CryptoInputStream.read(CryptoInputStream.java:183)
> 	at java.io.DataInputStream.readFully(DataInputStream.java:195)
> 	at org.apache.hadoop.hive.llap.LlapCacheAwareFs$CacheAwareInputStream.read(LlapCacheAwareFs.java:264)
> 	at java.io.DataInputStream.read(DataInputStream.java:149)
> 	at org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:102)
> 	at org.apache.parquet.io.DelegatingSeekableInputStream.readFullyHeapBuffer(DelegatingSeekableInputStream.java:127)
> 	at org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:91)
> 	at org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:1174)
> 	at org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:805)
> 	at org.apache.hadoop.hive.ql.io.parquet.vector.VectorizedParquetRecordReader.checkEndOfRowGroup(VectorizedParquetRecordReader.java:429)
> 	at org.apache.hadoop.hive.ql.io.parquet.vector.VectorizedParquetRecordReader.nextBatch(VectorizedParquetRecordReader.java:407)
> 	at org.apache.hadoop.hive.ql.io.parquet.vector.VectorizedParquetRecordReader.next(VectorizedParquetRecordReader.java:359)
> 	at org.apache.hadoop.hive.ql.io.parquet.vector.VectorizedParquetRecordReader.next(VectorizedParquetRecordReader.java:93)
> 	at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:361)
> 	at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:79)
> 	at org.apache.hadoop.hive.ql.io.HiveRecordReader.doNext(HiveRecordReader.java:33)
> 	at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.next(HiveContextAwareRecordReader.java:117)
> 	at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(TezGroupedSplitsInputFormat.java:151)
> 	at org.apache.tez.mapreduce.lib.MRReaderMapred.next(MRReaderMapred.java:116)
> 	at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:68)
> 	at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:426)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:267)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:250)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
> 	at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
> 	at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
> 	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:1730)
> 	at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
> 	at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
> 	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
> 	at org.apache.hadoop.hive.llap.daemon.impl.StatsRecordingThreadPool$WrappedCallable.call(StatsRecordingThreadPool.java:118)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 	at java.lang.Thread.run(Thread.java:748)
> {code}
> usually, streams have the enclosed stream as a field in order to have a reference for the time they want to close it, but here LlapCacheAwareFs$CacheAwareInputStream.read opens a DataInputStream which is never closed, only used in the read method, which is suspicious to me



--
This message was sent by Atlassian Jira
(v8.20.1#820001)