Skip to content
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

Merged
merged 13 commits into from
Feb 12, 2024

Conversation

horizonzy
Copy link
Member

@horizonzy horizonzy commented Jul 26, 2023

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.

-rw-r--r--@  1 horizon  staff   1.0G  7 26 17:47 f.log

1. Test DefaultEntryLogger.scanEntryLog (10x performance improvement)

    public static void main(String[] args) throws IOException {
        ServerConfiguration serverConfiguration = new ServerConfiguration();
        serverConfiguration.setLedgerDirNames(new String[]{"/Users/horizon/Downloads/bk1/bk-data"});
        DefaultEntryLogger defaultEntryLogger = new DefaultEntryLogger(serverConfiguration);
        long l = System.currentTimeMillis();
        AtomicInteger entryCount = new AtomicInteger();
        defaultEntryLogger.scanEntryLog(15, new EntryLogScanner() {
            @Override
            public boolean accept(long ledgerId) {
                return true;
            }

            @Override
            public void process(long ledgerId, long offset, ByteBuf entry) throws IOException {
                entryCount.incrementAndGet();
            }
        });
        long spend = System.currentTimeMillis() - l;
        System.out.println("Scan entry end, entry count: " + entryCount.get());
        System.out.println("Spend: " + spend);
    }

Before this pr:

Scan entry end, entry count: 17886769
Spend: 37273

After this pr:

Scan entry end, entry count: 17886769
Spend: 3363

2. Test DefaultEntryLogger.readFromLogChannel (30x performance improvement)

    public static void main(String[] args) throws IOException {
        ServerConfiguration serverConfiguration = new ServerConfiguration();
        serverConfiguration.setLedgerDirNames(new String[]{"/Users/horizon/Downloads/bk1/bk-data"});
        DefaultEntryLogger defaultEntryLogger = new DefaultEntryLogger(serverConfiguration);
        long l = System.currentTimeMillis();
        BufferedReadChannel channel = defaultEntryLogger.getChannelForLogId(15);
        int pos = 0;
        //Read total 1gb data from the BufferedReadChannel.
        long gb = 1024 * 1024 * 1024;
        ByteBuf byteBuf = ByteBufAllocator.DEFAULT.heapBuffer(10);
        while (pos < gb) {
            defaultEntryLogger.readFromLogChannel(15, channel, byteBuf, 0);
            pos += byteBuf.readableBytes();
            byteBuf.clear();
        }
        long spend = System.currentTimeMillis() - l;
        System.out.println("Spend: " + spend);
    }

Before this pr:

Spend: 69810

After this pr:

Spend: 2338

@dlg99
Copy link
Contributor

dlg99 commented Jul 26, 2023

Good idea, great results.

Can you please check that these "Bookie Tests" failures aren't related the change:

Error:  Errors: 
Error:  org.apache.bookkeeper.bookie.DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval
Error:    Run 1: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
Error:    Run 2: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
Error:    Run 3: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
[INFO] 
Error:  org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageTest.testEntriesOutOfOrderWithFlush
Error:    Run 1: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
Error:    Run 2: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
Error:    Run 3: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
[INFO] 
Warning:  Flakes: 
Warning:  InterleavedLedgerStorageTest.testConsistencyCheckConcurrentGC
[INFO]   Run 1: PASS
Error:    Run 2: expected:<0> but was:<8354>
Error:    Run 3: expected:<0> but was:<7255>
Error:    Run 4: expected:<0> but was:<7712>

I'll restart the job to see if these are flaps though I don't think these tests flapped before.

@horizonzy
Copy link
Member Author

Good idea, great results.

Can you please check that these "Bookie Tests" failures aren't related the change:

Error:  Errors: 
Error:  org.apache.bookkeeper.bookie.DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval
Error:    Run 1: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
Error:    Run 2: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
Error:    Run 3: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
[INFO] 
Error:  org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageTest.testEntriesOutOfOrderWithFlush
Error:    Run 1: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
Error:    Run 2: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
Error:    Run 3: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
[INFO] 
Warning:  Flakes: 
Warning:  InterleavedLedgerStorageTest.testConsistencyCheckConcurrentGC
[INFO]   Run 1: PASS
Error:    Run 2: expected:<0> but was:<8354>
Error:    Run 3: expected:<0> but was:<7255>
Error:    Run 4: expected:<0> but was:<7712>

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.

@horizonzy
Copy link
Member Author

Good idea, great results.

Can you please check that these "Bookie Tests" failures aren't related the change:

Error:  Errors: 
Error:  org.apache.bookkeeper.bookie.DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval
Error:    Run 1: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
Error:    Run 2: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
Error:    Run 3: DefaultEntryLogTest.testAppendLedgersMapOnCacheRemoval:1223 » IO Not all ledge...
[INFO] 
Error:  org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageTest.testEntriesOutOfOrderWithFlush
Error:    Run 1: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
Error:    Run 2: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
Error:    Run 3: DbLedgerStorageTest.testEntriesOutOfOrderWithFlush:400 » IO Bad entry read fro...
[INFO] 
Warning:  Flakes: 
Warning:  InterleavedLedgerStorageTest.testConsistencyCheckConcurrentGC
[INFO]   Run 1: PASS
Error:    Run 2: expected:<0> but was:<8354>
Error:    Run 3: expected:<0> but was:<7255>
Error:    Run 4: expected:<0> but was:<7712>

I'll restart the job to see if these are flaps though I don't think these tests flapped before.

Fixed.

Copy link
Contributor

@eolivelli eolivelli left a 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?

@Reidddddd
Copy link
Contributor

I feel Enrico's purpose is to require some method docs to explain why it needs the method name or type (anyway)

And i skimmed the changes, seems if (xxx instance of DefaultEntryLogger) will resolve it? no need to add this method

@horizonzy
Copy link
Member Author

And i skimmed the changes, seems if (xxx instance of DefaultEntryLogger) will resolve it? no need to add this method

Using instanceOf is not a good practice, It violates polymorphism.

@Reidddddd
Copy link
Contributor

And i skimmed the changes, seems if (xxx instance of DefaultEntryLogger) will resolve it? no need to add this method

Using instanceOf is not a good practice, It violates polymorphism.

compactingLogWriteDone is only a private method in TransactionalEntryLogCompactor.java, not a common method in AbstractLogCompactor.

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...

@horizonzy
Copy link
Member Author

And i skimmed the changes, seems if (xxx instance of DefaultEntryLogger) will resolve it? no need to add this method

Using instanceOf is not a good practice, It violates polymorphism.

compactingLogWriteDone is only a private method in TransactionalEntryLogCompactor.java, not a common method in AbstractLogCompactor.

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.

@horizonzy
Copy link
Member Author

I feel Enrico's purpose is to require some method docs to explain why it needs the method name or type (anyway)

And i skimmed the changes, seems if (xxx instance of DefaultEntryLogger) will resolve it? no need to add this method

Finally, take your suggestion, thanks.

@horizonzy horizonzy closed this Aug 2, 2023
@horizonzy horizonzy reopened this Aug 2, 2023
@Reidddddd
Copy link
Contributor

Reidddddd commented Aug 2, 2023

+1

@horizonzy
Copy link
Member Author

rerun failure checks

@hangc0276 hangc0276 added this to the 4.17.0 milestone Aug 9, 2023
Copy link
Contributor

@hangc0276 hangc0276 left a 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) {
Copy link
Contributor

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.

Copy link
Member Author

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;
Copy link
Member

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.

Copy link
Member Author

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.

Copy link
Member

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.

What do you mean?

Copy link
Member

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.

Copy link
Member Author

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.

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.

Copy link
Member Author

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.

@horizonzy
Copy link
Member Author

From the user report, they cherry-pick this commit, and they found that there is a problem about it.

Caused by: org.apache.bookkeeper.bookie.DefaultEntryLogger$EntryLookupException$MissingEntryException: pos 1027337410 (entry 135 for ledgerId 574) past end of entryLog 46

I will try to find the problem with them.

@horizonzy
Copy link
Member Author

From the user report, they cherry-pick this commit, and they found that there is a problem about it.

Caused by: org.apache.bookkeeper.bookie.DefaultEntryLogger$EntryLookupException$MissingEntryException: pos 1027337410 (entry 135 for ledgerId 574) past end of entryLog 46

I will try to find the problem with them.

The user config setEntryLogPerLedgerEnabled=true, so the defaultEntryLogger will write multi entry log file at the same time, so the sealed flag is not set correctly.

If the config setEntryLogPerLedgerEnabled=true, the sealed flag always be false.

@horizonzy
Copy link
Member Author

Great job! Would you please add a test to protect this change? Especially for the sealed flag

Done.

@Reidddddd
Copy link
Contributor

Hope to see this improvement in the next release (thumb up

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

Lgtm

@hangc0276
Copy link
Contributor

@merlimat @lhotari Please help take a look at this PR, thanks.

Comment on lines 75 to 84
public long size() throws IOException {
if (sealed) {
if (fileSize == -1) {
fileSize = validateAndGetFileChannel().size();
}
return fileSize;
} else {
return validateAndGetFileChannel().size();
}
}
Copy link
Member

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?

Copy link
Contributor

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

Copy link
Member

@lhotari lhotari Feb 5, 2024

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

/**
* The absolute position of the next write operation.
*/
protected volatile long position;
in BufferedChannel class. In the case that the instance is a BufferedChannel class, perhaps it could use that information?

Is the challenge in that case that the same file could have a BufferedReadChannel instance and a BufferedChannel instance at the same time?

Copy link
Member Author

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.

Comment on lines +95 to +97
public boolean isSealed(long logId) {
return logId != writingLogId && logId != writingCompactingLogId;
}
Copy link
Member

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"?

Copy link
Member Author

@horizonzy horizonzy Feb 10, 2024

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) {
Copy link
Contributor

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.

Copy link
Member Author

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.

@merlimat merlimat merged commit f5e4a98 into apache:master Feb 12, 2024
20 checks passed
gaozhangmin pushed a commit to gaozhangmin/bookkeeper that referenced this pull request Feb 20, 2024
gaozhangmin pushed a commit to gaozhangmin/bookkeeper that referenced this pull request Feb 20, 2024
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
* 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.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

9 participants