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-6758) [replication] The replication-executor should make sure the file that it is replicating is closed before declaring success on that file
Date Wed, 10 Oct 2012 21:33:05 GMT

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

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

bq. please let me know if I missed something or misunderstood your concern

Consider this scenario. First this runs:

bq. Path newPath = computeFilename();

Then with your patch we add this file in ZK during:

bq. i.preLogRoll(oldPath, newPath);

Now let's say HDFS becomes unavailable or the RS fails and never gets to this line:

bq. HLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf);

You end up with a log tracked in ZK that doesn't exist. This RS's queue will be recovered
by another RS that will eventually try to read from that non-existing file. My concern is
how we're going to treat that file.
                
> [replication] The replication-executor should make sure the file that it is replicating
is closed before declaring success on that file
> ---------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-6758
>                 URL: https://issues.apache.org/jira/browse/HBASE-6758
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Devaraj Das
>            Assignee: Devaraj Das
>            Priority: Critical
>             Fix For: 0.96.0
>
>         Attachments: 6758-1-0.92.patch, 6758-2-0.92.patch, 6758-trunk-1.patch, 6758-trunk-2.patch,
6758-trunk-3.patch, 6758-trunk-4.patch, TEST-org.apache.hadoop.hbase.replication.TestReplication.xml
>
>
> I have seen cases where the replication-executor would lose data to replicate since the
file hasn't been closed yet. Upon closing, the new data becomes visible. Before that happens
the ZK node shouldn't be deleted in ReplicationSourceManager.logPositionAndCleanOldLogs. Changes
need to be made in ReplicationSource.processEndOfFile as well (currentPath related).

--
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

Mime
View raw message