Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A553B10F93 for ; Thu, 7 Nov 2013 18:11:27 +0000 (UTC) Received: (qmail 80665 invoked by uid 500); 7 Nov 2013 18:11:26 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 80073 invoked by uid 500); 7 Nov 2013 18:11:20 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 79661 invoked by uid 99); 7 Nov 2013 18:11:18 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 07 Nov 2013 18:11:18 +0000 Date: Thu, 7 Nov 2013 18:11:18 +0000 (UTC) From: "Ted Yu (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-9818) NPE in HFileBlock#AbstractFSReader#readAtOffset MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-9818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13816191#comment-13816191 ] Ted Yu commented on HBASE-9818: ------------------------------- TestHFileBlock#testConcurrentReading shows how multiple FSReader's use the same FSDataInputStreamWrapper Another solution I can think of is to introduce ref counting for the streams FSDataInputStreamWrapper wraps. The FSDataInputStreamWrapper#close() would decrement the count and would close the underlying stream if count reaches 0. > NPE in HFileBlock#AbstractFSReader#readAtOffset > ----------------------------------------------- > > Key: HBASE-9818 > URL: https://issues.apache.org/jira/browse/HBASE-9818 > Project: HBase > Issue Type: Bug > Reporter: Jimmy Xiang > Assignee: Ted Yu > Attachments: 9818-trial.txt, 9818-v2.txt, 9818-v3.txt, 9818-v4.txt, 9818-v5.txt > > > HFileBlock#istream seems to be null. I was wondering should we hide FSDataInputStreamWrapper#useHBaseChecksum. > By the way, this happened when online schema change is enabled (encoding) > {noformat} > 2013-10-22 10:58:43,321 ERROR [RpcServer.handler=28,port=36020] regionserver.HRegionServer: > java.lang.NullPointerException > at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1200) > at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1436) > at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1318) > at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:359) > at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:254) > at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:503) > at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.reseekTo(HFileReaderV2.java:553) > at org.apache.hadoop.hbase.regionserver.StoreFileScanner.reseekAtOrAfter(StoreFileScanner.java:245) > at org.apache.hadoop.hbase.regionserver.StoreFileScanner.reseek(StoreFileScanner.java:166) > at org.apache.hadoop.hbase.regionserver.StoreFileScanner.enforceSeek(StoreFileScanner.java:361) > at org.apache.hadoop.hbase.regionserver.KeyValueHeap.pollRealKV(KeyValueHeap.java:336) > at org.apache.hadoop.hbase.regionserver.KeyValueHeap.generalizedSeek(KeyValueHeap.java:293) > at org.apache.hadoop.hbase.regionserver.KeyValueHeap.requestSeek(KeyValueHeap.java:258) > at org.apache.hadoop.hbase.regionserver.StoreScanner.reseek(StoreScanner.java:603) > at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:476) > at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:129) > at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.populateResult(HRegion.java:3546) > at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:3616) > at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:3494) > at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextRaw(HRegion.java:3485) > at org.apache.hadoop.hbase.regionserver.HRegionServer.scan(HRegionServer.java:3079) > at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:27022) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:1979) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:90) > at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.consumerLoop(SimpleRpcScheduler.java:160) > at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.access$000(SimpleRpcScheduler.java:38) > at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler$1.run(SimpleRpcScheduler.java:110) > at java.lang.Thread.run(Thread.java:724) > 2013-10-22 10:58:43,665 ERROR [RpcServer.handler=23,port=36020] regionserver.HRegionServer: > org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException: Expected nextCallSeq: 53438 But the nextCallSeq got from client: 53437; request=scanner_id: 1252577470624375060 number_of_rows: 100 close_scanner: false next_call_seq: 53437 > at org.apache.hadoop.hbase.regionserver.HRegionServer.scan(HRegionServer.java:3030) > at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:27022) > at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:1979) > at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:90) > at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.consumerLoop(SimpleRpcScheduler.java:160) > at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.access$000(SimpleRpcScheduler.java:38) > at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler$1.run(SimpleRpcScheduler.java:110) > at java.lang.Thread.run(Thread.java:724) > {noformat} -- This message was sent by Atlassian JIRA (v6.1#6144)