hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jean-Daniel Cryans (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HBASE-3515) [replication] ReplicationSource can miss a log after RS comes out of GC
Date Tue, 15 Mar 2011 00:36:29 GMT

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

Jean-Daniel Cryans commented on HBASE-3515:
-------------------------------------------

I'm thinking of a more radical way of solving this issue, considering that the problem is
that the RS is able to roll a log even tho we already lost our session, I'm thinking that
we should call fs.close() from inside HRS.abort() thus preventing any other call from reaching
HDFS. The downside is that it's going to make a big BOOOM and every call to close regions
will fail in the ugliest fashion.

> [replication] ReplicationSource can miss a log after RS comes out of GC
> -----------------------------------------------------------------------
>
>                 Key: HBASE-3515
>                 URL: https://issues.apache.org/jira/browse/HBASE-3515
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.90.0
>            Reporter: Jean-Daniel Cryans
>            Assignee: Jean-Daniel Cryans
>            Priority: Critical
>             Fix For: 0.90.2
>
>         Attachments: HBASE-3515.patch
>
>
> This is from Hudson build 1738, if a log is about to be rolled and the ZK connection
is already closed then the replication code will fail at adding the new log in ZK but the
log will still be rolled and it's possible that some edits will make it in.
> From the log:
> {quote}
> 2011-02-08 10:21:20,618 FATAL [RegionServer:0;vesta.apache.org,46117,1297160399378.logRoller]
regionserver.HRegionServer(1383):
>  ABORTING region server serverName=vesta.apache.org,46117,1297160399378, load=(requests=1525,
regions=12,
>  usedHeap=273, maxHeap=1244): Failed add log to list
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss
for 
>  /1/replication/rs/vesta.apache.org,46117,1297160399378/2/vesta.apache.org%3A46117.1297160480509
> ...
> 2011-02-08 10:21:22,444 DEBUG [MASTER_META_SERVER_OPERATIONS-vesta.apache.org:56008-0]
wal.HLogSplitter(258):
>  Splitting hlog 8 of 8: hdfs://localhost:55474/user/hudson/.logs/vesta.apache.org,46117,1297160399378/vesta.apache.org%3A46117.1297160480509,
length=0
> 2011-02-08 10:21:22,862 DEBUG [MASTER_META_SERVER_OPERATIONS-vesta.apache.org:56008-0]
wal.HLogSplitter(436):
>  Pushed=31 entries from hdfs://localhost:55474/user/hudson/.logs/vesta.apache.org,46117,1297160399378/vesta.apache.org%3A46117.1297160480509
> {quote}
> The easiest thing to do would be let the exception out and cancel the log roll.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message