-
Notifications
You must be signed in to change notification settings - Fork 3.4k
[HBASE-25536] Remove 0 length wal file from logQueue if it belongs to… #2908
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
🎊 +1 overall
This message was automatically generated. |
wchevreuil
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense to me. How difficult is it to add UTs testing this?
| private void handleEofException(IOException e) { | ||
| if ((e instanceof EOFException || e.getCause() instanceof EOFException) && | ||
| logQueue.size() > 1 && this.eofAutoRecovery) { | ||
| (source.isRecovered() || logQueue.size() > 1) && this.eofAutoRecovery) { |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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..
There was a problem hiding this comment.
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 !
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
Will add in next commit. |
|
🎊 +1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
gjacoby126
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 (non-binding)
|
This patch applies cleanly for branch-2 |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
🎊 +1 overall
This message was automatically generated. |
|
💔 -1 overall
This message was automatically generated. |
virajjasani
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Left small nit, looks good otherwise
| when(source.isPeerEnabled()).thenReturn(true); | ||
| ReplicationSourceWALReader reader = | ||
| new ReplicationSourceWALReader(fs, conf, queue, 0, getDummyFilter(), source); | ||
| reader.run(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: reader.start()?
virajjasani
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1
| reader.run(); | ||
| // ReplicationSourceWALReaderThread#handleEofException method will | ||
| // remove empty log from logQueue. | ||
| assertEquals(0, queue.size()); |
There was a problem hiding this comment.
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.
…ld sources (#2908) Signed-off-by: Wellington Chevreuil <[email protected]> Signed-off-by: Geoffrey Jacoby <[email protected]> Signed-off-by: Bharath Vissapragada <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…ld sources (#2908) Signed-off-by: Wellington Chevreuil <[email protected]> Signed-off-by: Geoffrey Jacoby <[email protected]> Signed-off-by: Bharath Vissapragada <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…ld sources (#2908) Signed-off-by: Wellington Chevreuil <[email protected]> Signed-off-by: Geoffrey Jacoby <[email protected]> Signed-off-by: Bharath Vissapragada <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…ld sources (#2908) Signed-off-by: Wellington Chevreuil <[email protected]> Signed-off-by: Geoffrey Jacoby <[email protected]> Signed-off-by: Bharath Vissapragada <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…ld sources (#2912) (#2908) Signed-off-by: Wellington Chevreuil <[email protected]> Signed-off-by: Geoffrey Jacoby <[email protected]> Signed-off-by: Bharath Vissapragada <[email protected]> Signed-off-by: Viraj Jasani <[email protected]>
…ld sources (apache#2908) Signed-off-by: Wellington Chevreuil <[email protected]> Signed-off-by: Geoffrey Jacoby <[email protected]> Signed-off-by: Bharath Vissapragada <[email protected]> Signed-off-by: Viraj Jasani <[email protected]> (cherry picked from commit ea1a9e9) Change-Id: Ifb5a34850ff75f4353fc1f0aa6f534897bd5458a
… old sources.