Hi, I am doing some benchmarks on BK, I see that from 4.4.0 to 4.5.0 there is something "slow" but I cannot understand what. I really hope that I am wrong.
I am working with writes, I will pass to reads once writes will be ok. My problem is both on latency (time for AddComplete callback to complete) and on overall throuput. Actually I have two distinct problems, but working on the first problem I found a performance regression. I know that talking about "slow" things it is an hard matter, so I will try do describe as much as possible all the aspects that I think are relevant. First problem: under certain load performance (latency+throughput) degrade too much Second problem: the first problem is more evident in 4.5.0 Let's describe my testcase and why I am worried. The bench issues a batch of asyncAddEntry and prints the average time for AddComplete to complete and the overall clock time. This is the code private static final byte[] TEST_DATA = new byte[35 * 1024]; private static final int testsize = 1000; ...... (start 1 bookie, see below) ClientConfiguration clientConfiguration = new ClientConfiguration(); clientConfiguration.setZkServers(env.getAddress()); try (BookKeeper bk = new BookKeeper(clientConfiguration); LedgerHandle lh = bk.createLedger(1, 1, 1, BookKeeper.DigestType.CRC32, new byte[0])) { LongAdder totalTime = new LongAdder(); long _start = System.currentTimeMillis(); Collection<CompletableFuture> batch = new ConcurrentLinkedQueue<>(); for (int i = 0; i < testsize; i++) { CompletableFuture cf = new CompletableFuture(); batch.add(cf); lh.asyncAddEntry(TEST_DATA, new AsyncCallback.AddCallback() { long start = System.currentTimeMillis(); @Override public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { long now = System.currentTimeMillis(); CompletableFuture _cf = (CompletableFuture) ctx; if (rc == BKException.Code.OK) { _cf.complete(""); } else { _cf.completeExceptionally(BKException.create(rc)); } totalTime.add(now - start); } }, cf); // Thread.sleep(1); // this is the tirgger!!! } assertEquals(testsize, batch.size()); for (CompletableFuture f : batch) { f.get(); } long _stop = System.currentTimeMillis(); long delta = _stop - _start; System.out.println("Total time: " + delta + " ms"); System.out.println("Total real time: " + totalTime.sum() + " ms -> "+(totalTime.sum()/testsize)+" ms per entry"); } Bookie config: ServerConfiguration conf = new ServerConfiguration(); conf.setBookiePort(5621); conf.setUseHostNameAsBookieID(true); Path targetDir = path.resolve("bookie_data"); conf.setZkServers("localhost:1282"); conf.setLedgerDirNames(new String[]{targetDir.toAbsolutePath().toString()}); conf.setJournalDirName(targetDir.toAbsolutePath().toString()); conf.setFlushInterval(1000); conf.setJournalFlushWhenQueueEmpty(true); conf.setProperty("journalMaxGroupWaitMSec", 0); conf.setProperty("journalBufferedWritesThreshold", 1024); conf.setAutoRecoveryDaemonEnabled(false); conf.setEnableLocalTransport(true); conf.setAllowLoopback(true); The tests starts one ZK server + 1 Bookie + the testcase in a JUnit test Results: A - BK-4.4.0: Total time: 209 ms Total real time: 194337 ms -> 194 ms per entry B - BK-4.5.0-SNAPSHOT: Total time: 269 ms Total real time: 239918 ms -> 239 ms per entry C - BK-4.4,0 with sleep(1): Total time: 1113 ms (1000 ms sleep time) Total real time: 4238 ms -> 4 ms per entry D - BK-4.5,0-SNAPSHOT with sleep(1): Total time: 1121 ms (1000 ms sleep time) Total real time: 8018 ms -> 8 ms per entry Problem 1 (unexpected performance degradation): Times per entry (latency) are incredibly slow in cases A and B. If I add a sleep(1) between one call of asyncAddEntry and the next "latency" is around 4 ms per entry. Problem 2: worse performance on 4.5.0 Compare A vs B and C vs D, it is self-explaining. I am running the test on my laptop, with linux 64bit (Fedora), 12 GB RAM, no swap, on an SSD disk. The results are similar on other computers. It seems that if I issue too many addEntry the systems slows down. Please note this fact: numbers for case A and B (without sleep) mean that all the adds got completed almost together for the 4.5 vs 4.4 case: I tried to disable all of the threadpool enhancements (different read/write pools)....it makes not difference Questions: Is the "grouping" logic of the journal ? Is there a way of making a burst of 1000 async writes on the same ledger perform <10 ms latency ? (in my real case I have bursts of concurrent writes from different threads) Why 4.5.0 is anyway slower ? Thanks -- Enrico