[ https://issues.apache.org/jira/browse/HBASE-6758?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13473005#comment-13473005 ]
Devaraj Das commented on HBASE-6758: ------------------------------------ Hey [~jdcryans], this patch doesn't change that behavior at all (new log is put up in ZK before the log is being written to, and blocks talking to ZK..). This patch only changes the postLogRoll placement and that deterministically ensures the previous log file is really closed before enqueuing the new log for replication. The code changes in the replicator thread (ReplicationSource.java) makes sure that the entire iteration of the loop "sees" a closed log file at least once (and hence takes care of the problem reported in the jira). > [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, > 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