Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -247,8 +247,10 @@ private void handleEmptyWALEntryBatch() throws InterruptedException {
// (highly likely we've closed the current log), we've hit the max retries, and autorecovery is
// enabled, then dump the log
private void handleEofException(IOException e) {
// Dump the log even if logQueue size is 1 if the source is from recovered Source
// since we don't add current log to recovered source queue so it is safe to remove.
if ((e instanceof EOFException || e.getCause() instanceof EOFException) &&
logQueue.size() > 1 && this.eofAutoRecovery) {
(source.isRecovered() || logQueue.size() > 1) && this.eofAutoRecovery) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice find, mind adding a quick comment since this is a subtle behavior?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

On a related note, should we add a source and global metric counter to track the number of 0 size files dequeued? Seems more common than we thought, easy to monitor may be..

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

On a related note, should we add a source and global metric counter to track the number of 0 size files dequeued? Seems more common than we thought, easy to monitor may be..

@bharathv Good idea. Could you please create a separate jira to track this. Thank you !

try {
if (fs.getFileStatus(logQueue.peek()).getLen() == 0) {
LOG.warn("Forcing removal of 0 length log in queue: " + logQueue.peek());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
Expand Down Expand Up @@ -652,4 +653,33 @@ public void testReadBeyondCommittedLength() throws IOException, InterruptedExcep
assertFalse(entryStream.hasNext());
}
}

/*
Test removal of 0 length log from logQueue if the source is a recovered source and
size of logQueue is only 1.
*/
@Test
public void testEOFExceptionForRecoveredQueue() throws Exception {
PriorityBlockingQueue<Path> queue = new PriorityBlockingQueue<>();
// Create a 0 length log.
Path emptyLog = new Path("emptyLog");
FSDataOutputStream fsdos = fs.create(emptyLog);
fsdos.close();
assertEquals(0, fs.getFileStatus(emptyLog).getLen());
queue.add(emptyLog);

Configuration conf = new Configuration(CONF);
// Override the max retries multiplier to fail fast.
conf.setInt("replication.source.maxretriesmultiplier", 1);
conf.setBoolean("replication.source.eof.autorecovery", true);
// Create a reader thread with source as recovered source.
ReplicationSource source = mockReplicationSource(true, conf);
when(source.isPeerEnabled()).thenReturn(true);
ReplicationSourceWALReader reader =
new ReplicationSourceWALReader(fs, conf, queue, 0, getDummyFilter(), source);
reader.run();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: reader.start()?

// ReplicationSourceWALReaderThread#handleEofException method will
// remove empty log from logQueue.
assertEquals(0, queue.size());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see, If we use thread.start(), this could be flaky if we reach here before reader thread removes entry from queue so we will need to add some wait here based on this exit condition (queue.isEmpty()). It's a trade off with starting thread separately as opposed to synchronously executing it with thread.run().

It's upto you, this is anyways trivial. thread.run() is also fine.

}
}