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 CB0F2E213 for ; Tue, 29 Jan 2013 21:07:13 +0000 (UTC) Received: (qmail 33731 invoked by uid 500); 29 Jan 2013 21:07:13 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 33690 invoked by uid 500); 29 Jan 2013 21:07:13 -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 33681 invoked by uid 99); 29 Jan 2013 21:07:13 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Jan 2013 21:07:13 +0000 Date: Tue, 29 Jan 2013 21:07:13 +0000 (UTC) From: "Matteo Bertozzi (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-7711) rowlock release problem with thread interruptions in batchMutate 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-7711?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13565805#comment-13565805 ] Matteo Bertozzi commented on HBASE-7711: ---------------------------------------- Should we use something like, to preserve the stack trace? {code} InterruptedIOException ie = new InterruptedIOException(e.getMessage()); ie.initCause(e); {code} > rowlock release problem with thread interruptions in batchMutate > ---------------------------------------------------------------- > > Key: HBASE-7711 > URL: https://issues.apache.org/jira/browse/HBASE-7711 > Project: HBase > Issue Type: Bug > Reporter: Jonathan Hsieh > Assignee: Ted Yu > Fix For: 0.96.0 > > Attachments: 7711.txt > > > An earlier version of snapshots would thread interrupt operations. In longer term testing we ran into an exception stack trace that indicated that a rowlock was taken an never released. > {code} > 2013-01-26 01:54:56,417 ERROR org.apache.hadoop.hbase.procedure.ProcedureMember: Propagating foreign exception to subprocedure pe-1 > org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable via timer-java.util.Timer@1cea3151:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign E > xception Start:1359194035004, End:1359194095004, diff:60000, max:60000 ms > at org.apache.hadoop.hbase.errorhandling.ForeignException.deserialize(ForeignException.java:184) > at org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.abort(ZKProcedureMemberRpcs.java:321) > at org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.watchForAbortedProcedures(ZKProcedureMemberRpcs.java:150) > at org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.access$200(ZKProcedureMemberRpcs.java:56) > at org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs$1.nodeChildrenChanged(ZKProcedureMemberRpcs.java:112) > at org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:315) > at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:519) > at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:495) > Caused by: org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1359194035004, End:1359194095004, diff:60000, max:60000 ms > at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71) > at java.util.TimerThread.mainLoop(Timer.java:512) > at java.util.TimerThread.run(Timer.java:462) > 2013-01-26 01:54:56,648 WARN org.apache.hadoop.hbase.regionserver.HRegion: Failed getting lock in batch put, row=0001558252 > java.io.IOException: Timed out on getting lock for row=0001558252 > at org.apache.hadoop.hbase.regionserver.HRegion.internalObtainRowLock(HRegion.java:3239) > at org.apache.hadoop.hbase.regionserver.HRegion.getLock(HRegion.java:3315) > at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2150) > at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2021) > at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3511) > at sun.reflect.GeneratedMethodAccessor46.invoke(Unknown Source) > at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at org.apache.hadoop.hbase.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:364) > at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1400) > .... > .. every snapshot attempt that used this region for the next two days encountered this problem. > {code} > Snapshots will now bypass this problem with the fix in HBASE-7703. However, we should make sure hbase regionserver operations are safe when interrupted. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira