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

Duo Zhang commented on HBASE-19792:
-----------------------------------

OK there is a critical bug which may leads to data inconsistency when 
replication.

This is the log.
{noformat}
2018-01-15 03:13:16,649 INFO  [asf911:59564Replication Statistics #0] 
regionserver.Replication$ReplicationStatisticsThread(352): Normal source for 
cluster 2: Total replicated edits: 4, current progress:
walGroup [asf911.gq1.ygridcore.net%2C59564%2C1515985899707]: currently 
replicating from: 
/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/0eb32823-c0e3-4ebf-af26-f6fd89e2cbdd/asf911.gq1.ygridcore.net%2C59564%2C1515985899707.1515985992379
 at position: 4871


2018-01-15 03:13:16,649 INFO  [asf911:56726Replication Statistics #0] 
regionserver.Replication$ReplicationStatisticsThread(352): Normal source for 
cluster 2: Total replicated edits: 1003, current progress:
walGroup [asf911.gq1.ygridcore.net%2C56726%2C1515985899619]: currently 
replicating from: 
/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/0eb32823-c0e3-4ebf-af26-f6fd89e2cbdd/asf911.gq1.ygridcore.net%2C56726%2C1515985899619.1515985992379
 at position: 122068


2018-01-15 03:13:16,937 DEBUG 
[RS_REFRESH_PEER-asf911:59564-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C59564%2C1515985899707,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C59564%2C1515985899707,2]
 regionserver.ReplicationSourceWALReader(153): Failed to read stream of 
replication entries: java.io.EOFException: Cannot seek after EOF
2018-01-15 03:13:16,937 DEBUG 
[RS_REFRESH_PEER-asf911:56726-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C56726%2C1515985899619,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C56726%2C1515985899619,2]
 regionserver.ReplicationSourceWALReader(153): Failed to read stream of 
replication entries: java.io.EOFException: Cannot seek after EOF
2018-01-15 03:13:17,940 ERROR 
[RS_REFRESH_PEER-asf911:59564-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C59564%2C1515985899707,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C59564%2C1515985899707,2]
 regionserver.ReplicationSourceWALReader(156): Failed to read stream of 
replication entries
java.io.EOFException: Cannot seek after EOF
        at org.apache.hadoop.hdfs.DFSInputStream.seek(DFSInputStream.java:1516)
        at 
org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:62)
        at 
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.initInternal(ProtobufLogReader.java:209)
        at 
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.initReader(ProtobufLogReader.java:171)
        at 
org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:65)
        at 
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
        at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
        at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-15 03:13:17,940 ERROR 
[RS_REFRESH_PEER-asf911:56726-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C56726%2C1515985899619,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C56726%2C1515985899619,2]
 regionserver.ReplicationSourceWALReader(156): Failed to read stream of 
replication entries
java.io.EOFException: Cannot seek after EOF
        at org.apache.hadoop.hdfs.DFSInputStream.seek(DFSInputStream.java:1516)
        at 
org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:62)
        at 
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.initInternal(ProtobufLogReader.java:209)
        at 
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.initReader(ProtobufLogReader.java:171)
        at 
org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:65)
        at 
org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
        at 
org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
        at 
org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
        at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
        at 
org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-15 03:13:17,941 WARN  
[RS_REFRESH_PEER-asf911:59564-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C59564%2C1515985899707,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C59564%2C1515985899707,2]
 regionserver.ReplicationSourceWALReader(211): Forcing removal of 0 length log 
in queue: 
/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/0eb32823-c0e3-4ebf-af26-f6fd89e2cbdd/asf911.gq1.ygridcore.net%2C59564%2C1515985899707.1515985992379
2018-01-15 03:13:17,941 WARN  
[RS_REFRESH_PEER-asf911:56726-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C56726%2C1515985899619,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C56726%2C1515985899619,2]
 regionserver.ReplicationSourceWALReader(211): Forcing removal of 0 length log 
in queue: 
/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/0eb32823-c0e3-4ebf-af26-f6fd89e2cbdd/asf911.gq1.ygridcore.net%2C56726%2C1515985899619.1515985992379
{noformat}

You can see that, the logs are not zero-length, we have actually replicated 
something. And then we hit EOF, and then there is a force removal which says 
the files are both zero-length.

The code is like this:

{code}
    if (e instanceof EOFException ||
        e.getCause() instanceof EOFException && logQueue.size() > 1 && 
this.eofAutoRecovery) {
      try {
        if (fs.getFileStatus(logQueue.peek()).getLen() == 0) {
          LOG.warn("Forcing removal of 0 length log in queue: " + 
logQueue.peek());
          logQueue.remove();
          currentPosition = 0;
        }
      } catch (IOException ioe) {
        LOG.warn("Couldn't get file length information about log " + 
logQueue.peek());
      }
    }
{code}

The immediate cause here is that, these files are all opened for writing now so 
the getFileStatus could return with a zero length if there is only one block, 
and then we removed the files from log queue. Still not sure why this could 
lead to the FNFE later but we need to fix this...

I do not think the author want this happen since there is a 'logQueue.size() > 
1' but I think the priority of the operators causes the problem... We missed a 
pair of '()'... Let me fix this issue first and then see whether the test still 
fails.

> TestReplicationSmallTests.testDisableEnable fails
> -------------------------------------------------
>
>                 Key: HBASE-19792
>                 URL: https://issues.apache.org/jira/browse/HBASE-19792
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Duo Zhang
>            Priority: Major
>             Fix For: 2.0.0-beta-2
>
>         Attachments: 
> org.apache.hadoop.hbase.replication.TestReplicationSmallTests-output.txt
>
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to