You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "James Moore (JIRA)" <ji...@apache.org> on 2017/01/24 21:12:26 UTC

[jira] [Commented] (HBASE-17501) NullPointerException after Datanodes Decommissioned and Terminated

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

James Moore commented on HBASE-17501:
-------------------------------------

Hi stack,

I work with Pat on the HBase team over at HubSpot.  

It looks like we end up with the server locked due to this code in HFileBlock, along with an access pattern which only hits that block via this code path.  In the event, the blockReader is set to null the region server will currently never have the opportunity to re-open it.

{code}     if (!pread && streamLock.tryLock()) {  
        // Seek + read. Better for scanning.
        try {
          //NPE is thrown
          istream.seek(fileOffset);

          long realOffset = istream.getPos();
          if (realOffset != fileOffset) {
            throw new IOException("Tried to seek to " + fileOffset + " to "
                + "read " + size + " bytes, but pos=" + realOffset
                + " after seek");
          }

          if (!peekIntoNextBlock) {
            // the blockReader from the DFSInputStream is nulled
            // and a new instance is created in this method.
            IOUtils.readFully(istream, dest, destOffset, size);
            return -1;
          }

          // Try to read the next block header.
          if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
            return -1;
        } finally {
          streamLock.unlock();
        }
{code}

Our logs indicate that there were a large number of exceptions triggered during the construction of the blockReader such as the following.

{quote} WARN org.apache.hadoop.hdfs.BlockReaderFactory: I/O error constructing remote block reader.
org.apache.hadoop.net.ConnectTimeoutException: 10000 millis timeout while waiting for channel to be ready for connect. ch : java.nio.channels.SocketChannel[con
nection-pending remote=ourhostname/ourip:50010]
        at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:533)
        at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3519)
        at org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:840)
        at org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:755)
        at org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:376)
        at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1174)
        at org.apache.hadoop.hdfs.DFSInputStream.access$300(DFSInputStream.java:91)
        at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1136)
        at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1128)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        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)
{quote}

It looks like the DFSInputStream is holding onto block locations that include the decommissioned data nodes, and potentially builds a null client when the IOException occurs while constructing a remoteBlockReader from [peerCache|https://github.com/cloudera/hadoop-common/blob/cdh5-2.6.0_5.9.0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java#L769-L775 ]

its unclear from our logs which exact state caused the blockReader to become null. However, we can see repeated attempts to construct blockReaders to the non-existent decommissioned data nodes.

> NullPointerException after Datanodes Decommissioned and Terminated
> ------------------------------------------------------------------
>
>                 Key: HBASE-17501
>                 URL: https://issues.apache.org/jira/browse/HBASE-17501
>             Project: HBase
>          Issue Type: Bug
>         Environment: CentOS Derivative with a derivative of the 3.18.43 kernel.  HBase on CDH5.9.0 with some patches.  HDFS CDH 5.9.0 with no patches.
>            Reporter: Patrick Dignan
>            Priority: Minor
>
> We recently encountered an interesting NullPointerException in HDFS that bubbles up to HBase, and is resolved be restarting the regionserver.  The issue was exhibited while we were replacing a set of nodes in one of our clusters with a new set.  We did the following:
> 1. Turn off the HBase balancer
> 2. Gracefully move the regions off the nodes we’re shutting off using a tool we wrote to do so
> 3. Decommission the datanodes using the HDFS exclude hosts file and hdfs dfsadmin -refreshNodes
> 4. Wait for the datanodes to decommission fully
> 5. Terminate the VMs the instances are running inside.
> A few notes.  We did not shutdown the datanode processes, and the nodes were therefore not marked as dead by the namenode.  We simply terminated the datanode VM (in this case an AWS instance).  The nodes were marked as decommissioned.  We are running our clusters with DNS, and when we terminate VMs, the associated CName is removed and no longer resolves.  The errors do not seem to resolve without a restart.
> After we did this, the remaining regionservers started throwing NullPointerExceptions with the following stack trace:
> 2017-01-19 23:09:05,638 DEBUG org.apache.hadoop.hbase.ipc.RpcServer: RpcServer.RW.fifo.Q.read.handler=80,queue=14,port=60020: callId: 1727723891 service: ClientService methodName: Scan size: 216 connection: 172.16.36.128:31538
> java.io.IOException
>     at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2214)
>     at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:123)
>     at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:204)
>     at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:183)
> Caused by: java.lang.NullPointerException
>     at org.apache.hadoop.hdfs.DFSInputStream.seek(DFSInputStream.java:1564)
>     at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:62)
>     at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1434)
>     at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1682)
>     at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1542)
>     at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:445)
>     at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:266)
>     at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:642)
>     at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:592)
>     at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:294)
>     at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:199)
>     at org.apache.hadoop.hbase.regionserver.StoreScanner.seekScanners(StoreScanner.java:343)
>     at org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:198)
>     at org.apache.hadoop.hbase.regionserver.HStore.createScanner(HStore.java:2106)
>     at org.apache.hadoop.hbase.regionserver.HStore.getScanner(HStore.java:2096)
>     at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:5544)
>     at org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:2569)
>     at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2555)
>     at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2536)
>     at org.apache.hadoop.hbase.regionserver.RSRpcServices.scan(RSRpcServices.java:2405)
>     at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:33738)
>     at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2170)
>     ... 3 more



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