-
Notifications
You must be signed in to change notification settings - Fork 902
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
Improve DefaultEntryLogger read performance. #4038
Improve DefaultEntryLogger read performance. #4038
Conversation
Good idea, great results. Can you please check that these "Bookie Tests" failures aren't related the change:
I'll restart the job to see if these are flaps though I don't think these tests flapped before. |
I have checked the testAppendLedgersMapOnCacheRemoval and testEntriesOutOfOrderWithFlush. The problem is that the BufferedReadChannel and BufferedLogChannel share the same FileChannel. When the BufferedReadChannel finishes reading the fileSize, the BufferedLogChannel writes new data into the file. I will verify if this situation exists. In theory, the FileChannel read by the BufferedLogChannel should not write any new data. |
Fixed. |
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.
Overall I support this change.
But I am not sure about the 'name' name.
Can we fins something that explains better the purpose of this property?
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogger.java
Outdated
Show resolved
Hide resolved
I feel Enrico's purpose is to require some method docs to explain why it needs the method And i skimmed the changes, seems |
Using instanceOf is not a good practice, It violates polymorphism. |
What you did, in order to serve a private function in a concrete sub class, you added a function and parameters for other classes which don't need them When you are talking about polymorphism, polymorphism allows each sub-class has its own behaviour, you are enforcing all the sub-class to satisfy a concrete need not a common need, you are the one who breaks polymorphism... |
Ok, that makes sense. I would like to public an interface to do the work. |
Finally, take your suggestion, thanks. |
+1 |
rerun failure checks |
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.
Great job! Would you please add a test to protect this change? Especially for the sealed flag
|
||
public BufferedReadChannel(FileChannel fileChannel, int readCapacity) { | ||
public BufferedReadChannel(FileChannel fileChannel, int readCapacity, boolean sealed) { |
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.
Do we need to keep the original constructor method and create a new one with sealed
flag? Because the BufferedReadChannel
class is public and the constructor is also public, other applications may use this channel to create instances. If we change the constructor directly, it will break the compatibility with the old versions.
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.
Make sense.
} | ||
} | ||
|
||
void clearCompactingLogId() { | ||
writingCompactingLogId = -1; |
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.
Do we need to guard this by synchronized (createCompactionLogLock)
? It seems not possible to have race conditions, but I see the createNewLogForCompaction(File dirForNextEntryLog)
is guarded.
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.
No need this, the createNewLogForCompaction
and clearCompactingLogId
are serial.
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.
No need this, the
createNewLogForCompaction
andclearCompactingLogId
are serial.
What do you mean?
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.
Just wondering if there's a chance that writingLogId and writingCompactingLogId field changes wouldn't be visible for another thread that reads these fields.
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.
No need this, the
createNewLogForCompaction
andclearCompactingLogId
are serial.What do you mean?
The method createNewLogForCompaction
is synchronized. It creates compact file one by one, so the previous log file id can't override the next log file id.
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.
Just wondering if there's a chance that writingLogId and writingCompactingLogId field changes wouldn't be visible for another thread that reads these fields.
Make sense, make it volatile.
From the user report, they cherry-pick this commit, and they found that there is a problem about it.
I will try to find the problem with them. |
The user config If the config |
Done. |
Hope to see this improvement in the next release (thumb up |
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.
Lgtm
public long size() throws IOException { | ||
if (sealed) { | ||
if (fileSize == -1) { | ||
fileSize = validateAndGetFileChannel().size(); | ||
} | ||
return fileSize; | ||
} else { | ||
return validateAndGetFileChannel().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.
Would there be a benefit to override this logic in BufferedChannel
class where the position
(and possibly unpersistedBytes
) field information would be leveraged? Doesn't position
contain the file size when it's writable?
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.
The FileChnanel doesn't have position
field and the position()
method will trigger the OS system call
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.
@hangc0276 I was referring to
bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BufferedChannel.java
Lines 49 to 52 in 2209734
/** | |
* The absolute position of the next write operation. | |
*/ | |
protected volatile long position; |
Is the challenge in that case that the same file could have a BufferedReadChannel instance and a BufferedChannel instance at the same time?
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'm not got you fully, the BufferedChannel means the corresponding file is writing, it override the BufferedReadChannel#read
method, it know the file end position.
Is the challenge in that case that the same file could have a BufferedReadChannel instance and a BufferedChannel instance at the same time?
#4038 (comment)
The test contains the case.
public boolean isSealed(long logId) { | ||
return logId != writingLogId && logId != writingCompactingLogId; | ||
} |
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.
Isn't this misleading if it is called "isSealed"?
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.
isSealed
means that the entry log file will no longer be written to, we needn't care that the file size change.
@Override | ||
public long size() throws IOException { | ||
if (sealed) { | ||
if (fileSize == -1) { |
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: if the file is already "sealed", should we read the size in the constructor? This way we wouldn't have any concurrency issues here if 2 threads call size()
at the size time.
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.
The size()
would throw IOException, I don't want to make the constructor announce the exception, so make the fileSize double check to avoid concurrency.
* Avoid system call to improve read performance. * Fix ci. * Add comments for getCurrentWritingLogId * Fix ci. * Consider compacting log. * Fix checkstyle. * Address the comment. * Address comment. * Address the comments. * Add tests. * Fix checkstyle. * address the comments. * Fix concurrency problem.
If the BufferChannel is BufferedReadChannel, the size won't change. We can cache the fileSize to avoid time-consuming operation.
Performance tests.
The entry log
f
file holds 1GB data.1. Test DefaultEntryLogger.scanEntryLog (10x performance improvement)
Before this pr:
After this pr:
2. Test DefaultEntryLogger.readFromLogChannel (30x performance improvement)
Before this pr:
After this pr: