billoley opened a new issue, #3617:
URL: https://github.com/apache/accumulo/issues/3617
Running Accumulo 2.1.1, Hadoop 3.3.3 on an AWS test system.
When an IOException (InterruptedIOException in my case) is caught in
RFile$LocalityGroupReader.seek or RFile$LocalityGroupReader.next, the
RFile$LocalityGroupReader calls RFile$LocalityGroupReader.reset() to close
currBlock and set it to null but CachableBlockFile$Reader.bcfr still holds a
reference to SeekableDataInputStream below the point that close descends to.
currBlock (created by CachableBlockFile$Reader.getDataBlock()) contains the
following nested streams
BufferedInputStream gets closed
DecompressorStream gets closed
BoundedRangeFileInputStream sets closed to true
SeekableDataInputStream <-- this is the stream held by
CachableBlockFile$Reader.bcfr
RateLimitedInputStream
HdfsDataInputStream
DFSInputStream -- when an HDFS read is interrupted while fetching a
blockReader, blockReader can remain null which causes an NPE when this stream
gets reused.
CachableBlockFile$Reader gets reclaimed from and re-assigned to
ScanDataSources in a Tablet's FileManager.
InterruptedIOException during HDFS read
```
java.io.InterruptedIOException: Interrupted while waiting for IO on channel
java.nio.channels.SocketChannel[connected local=/127.0.0.1:50444
remote=/127.0.0.1:9866]. Total timeout mills is 60000, 60000 millis timeout
left.
at
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:350)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:156)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118)
~[hadoop-client-api-3.3.3.jar:?]
at java.io.FilterInputStream.read(FilterInputStream.java:83) ~[?:?]
at
org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed(PBHelperClient.java:519)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.newBlockReader(BlockReaderRemote.java:414)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReader(BlockReaderFactory.java:861)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:757)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:381)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:755)
~[hadoop-client-api-3.3.3.jar:?]
>>> at
org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:685)
~[hadoop-client-api-3.3.3.jar:?] <<<<<<< leaves blockReader == null
at
org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:884)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:957)
~[hadoop-client-api-3.3.3.jar:?]
at java.io.DataInputStream.read(DataInputStream.java:149) ~[?:?]
at
org.apache.accumulo.core.file.streams.RateLimitedInputStream.read(RateLimitedInputStream.java:52)
~[accumulo-core-2.1.1.jar:2.1.1]
at java.io.DataInputStream.read(DataInputStream.java:149) ~[?:?]
at
org.apache.accumulo.core.file.streams.BoundedRangeFileInputStream.read(BoundedRangeFileInputStream.java:98)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.hadoop.io.compress.DecompressorStream.getCompressedData(DecompressorStream.java:179)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:163)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:105)
~[hadoop-client-api-3.3.3.jar:?]
at java.io.BufferedInputStream.fill(BufferedInputStream.java:252)
~[?:?]
at java.io.BufferedInputStream.read(BufferedInputStream.java:271)
~[?:?]
at java.io.FilterInputStream.read(FilterInputStream.java:83) ~[?:?]
at java.io.DataInputStream.readByte(DataInputStream.java:270) ~[?:?]
at
org.apache.accumulo.core.file.rfile.RelativeKey.fastSkip(RelativeKey.java:296)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader._seek(RFile.java:1075)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader.seek(RFile.java:915)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator._seek(LocalityGroupIterator.java:167)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.seek(LocalityGroupIterator.java:276)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.file.rfile.RFile$Reader.seek(RFile.java:1469)
~[accumulo-core-2.1.1.jar:2.1.1]
```
IOException caught in RFile$LocalityGroupReader and
RFile$LocalityGroupReader.reset() called, but CachableBlockFile$Reader.bcfr
still has a reference to the SeekableDataInputStream. It gets saved off and
then reused resulting in a NPE
Saving the Readers after ScanDataSource is done being used
```
Scanner.read() in finally block calls
ScanDataSource.detachFileManager() calls
FileManager$ScanFileManager.detach() calls
FileManager.releaseReaders()
```
Re-assigning RFile$Reader from openFiles to a newly created ScanDataSource
```
Scanner.read() calls
TabletBase.createDataSource() created new
ScanDataSource() and creates new SourceSwitchingIterator() with it, calls
tablet.nextBatch() which sets the yield and then calls
SourceSwitchingIterator.seek() calls
SourceSwitchingIterator.iterator() implemented by
ScanDataSource.iterator() calls
ScanDataSource.createIterator()
Creates new ScanFileManager calls
FileManager.openFiles() calls
FileManager.openFiles() calls
FileManager.reserveReaders() calls
FileManager.takeOpenFiles()
Which copies RFile$Reader instances from openFiles to reservedReaders
(RFile$Reader contains an array of RFile$LocalityGroupReader each of which
contains a CachableBlockFile.Reader)
```
NullPointerException in DFSInputStream (in this case)
```
java.lang.NullPointerException: null
at
org.apache.hadoop.hdfs.DFSInputStream.seek(DFSInputStream.java:1614)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:71)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.accumulo.core.file.streams.RateLimitedInputStream.seek(RateLimitedInputStream.java:61)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.file.streams.SeekableDataInputStream.seek(SeekableDataInputStream.java:38)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.file.streams.BoundedRangeFileInputStream.read(BoundedRangeFileInputStream.java:97)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.hadoop.io.compress.DecompressorStream.getCompressedData(DecompressorStream.java:179)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:163)
~[hadoop-client-api-3.3.3.jar:?]
at
org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:105)
~[hadoop-client-api-3.3.3.jar:?]
at java.io.BufferedInputStream.fill(BufferedInputStream.java:252)
~[?:?]
at java.io.BufferedInputStream.read(BufferedInputStream.java:271)
~[?:?]
at java.io.FilterInputStream.read(FilterInputStream.java:83) ~[?:?]
at java.io.DataInputStream.readByte(DataInputStream.java:270) ~[?:?]
at
org.apache.accumulo.core.file.rfile.RelativeKey.fastSkip(RelativeKey.java:296)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader._seek(RFile.java:1075)
~[accumulo-core-2.1.1.jar:2.1.1]
at
org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader.seek(RFile.java:915)
~[accumulo-core-2.1.1.jar:2.1.1]
```
A possible solution is that RFile$LocalityGroupReader.reset() besides
currBlock.close() and setting currBlock to null
should also call reader.close() – which should additionally reset the bcfr
AtomicReference to null.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]