On Wed, Jul 12, 2017 at 2:04 AM, Venkateswara Rao Jujjuri <jujj...@gmail.com > wrote:
> Enrico, let me try to paraphrase the issue. > > - With G1GC + Netty 4.1 is giving you roughly same perf as prev release. Is > that accurate statement? > But you are still seeing latency spikes with Netty 4.1?? > I did not fully > understand your sleep usecase. > How sleep is yielding better latency? > I think the "sleep(1)" in Enrico's benchmark is for throttling. It is same as setting 'throttle' value to 1000 in his test. - Sijie > > Thanks, > JV > > On Tue, Jul 11, 2017 at 8:27 AM, Enrico Olivelli <eolive...@gmail.com> > wrote: > > > Another interesting thing...during my profiling activity I gave a chance > to > > the old v2 protocol and activated the gc logs, as expected the result is > > that with v2 protocol there is almost no GC activity during the benchmark > > -- Enrico > > > > 2017-07-11 12:07 GMT+02:00 Enrico Olivelli <eolive...@gmail.com>: > > > > > > > > > > > 2017-07-11 11:04 GMT+02:00 Sijie Guo <guosi...@gmail.com>: > > > > > >> I think Netty4 requires more offheap memory. you might need to tune > the > > >> JVM > > >> settings. I doubt that latency diff coming from the JVM gc. > > >> > > >> A simple thing to verify that is to dump the gc log by adding " > > -Xloggc:" > > >> setting and compare the gc logs between versions. > > >> > > > > > > > > > Enabling G1 GC (-XX:+UseG1GC) apparently solves the difference between > > 4.4 > > > and 4.5. > > > > > > And I a loop the bench (in order to have a warm JVM) numbers become > more > > > stable and similar to each other > > > > > > I have the "other issue" pending, the fact that latency (time for > > > asynchAddEntry to complete) is really bad and unpredictable (from 100 > ms > > to > > > 2000ms) > > > > > > and if I introduce a Thread.sleep(1) all the callbacks complete always > > > with success in < 10ms. This happens even with G1 GC. > > > > > > Without the "sleep(1)" my machine uses a lot of CPU (I have 8 "CPUs") > and > > > with the "sleep(1)" the load is slightly lower > > > > > > Honestly for me this is still a problem and I hope that with your help > I > > > will be able to find the problem, wherever it is (in BK code or in the > > way > > > I am doing the bench) > > > > > > I will try to create a new more complete bench > > > > > > -- Enrico > > > > > > > > > > > > > > >> > > >> - Sijie > > >> > > >> On Tue, Jul 11, 2017 at 12:16 AM, Enrico Olivelli < > eolive...@gmail.com> > > >> wrote: > > >> > > >> > a did a bisect and the culprit (in my opinion) is the switch to > netty > > 4 > > >> for > > >> > the performance regression from 4.5 and 4.4 > > >> > > > >> > at commit: > > >> > commit 811ece53a1c975c4e768422f3d622ac9de6b3e41 BOOKKEEPER-1058: > > >> Ignore > > >> > already deleted ledger on replication audit > > >> > > > >> > Total time: 204 ms > > >> > Total real time: 79 ms per entry > > >> > > > >> > at commit: > > >> > commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008: > > Netty > > >> 4.1 > > >> > > > >> > Total time: 308 ms > > >> > Total real time: 189 ms per entry > > >> > > > >> > I have tried with epoll and with local transport, results does not > > >> change. > > >> > I tried to upgrade to netty 4.1.13 too, but no change > > >> > > > >> > Could it be the memory allocator of netty which is overwhelmed with > > >> sudden > > >> > bursts of allocation ? > > >> > I did some trial with UnpooledByteBufAllocator.DEFAULT and it > helps a > > >> > little, we get to "110 ms per entry" vs "189 ms per entry" > > >> > > > >> > the bench is here: > > >> > https://github.com/eolivelli/bookkeepers-benchs/blob/master/ > > >> src/test/java/ > > >> > BookKeeperWriteTest.java > > >> > > > >> > > > >> > -- Enrico > > >> > > > >> > > > >> > > > >> > 2017-07-10 19:46 GMT+02:00 Enrico Olivelli <eolive...@gmail.com>: > > >> > > > >> > > > > >> > > > > >> > > Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri < > > jujj...@gmail.com > > >> > > > >> > ha > > >> > > scritto: > > >> > > > > >> > >> With Netty changes, lack of native epoll() has huge perf impact > as > > >> per > > >> > >> Kishore. > > >> > >> Are you sure you are using epoll()? > > >> > >> > > >> > > > > >> > > Yes. I tried with netty local transport too. It seems not related > to > > >> > netty > > >> > > to me. > > >> > > I will double check, tomorrow > > >> > > Enrico > > >> > > > > >> > > > > >> > >> On Mon, Jul 10, 2017 at 1:49 AM, Enrico Olivelli < > > >> eolive...@gmail.com> > > >> > >> wrote: > > >> > >> > > >> > >> > 2017-07-10 10:40 GMT+02:00 Sijie Guo <guosi...@gmail.com>: > > >> > >> > > > >> > >> > > Also one other thing to check is the JVM settings. Do you > mind > > >> > sharing > > >> > >> > that > > >> > >> > > as well? > > >> > >> > > > > >> > >> > > > > >> > >> > this is the surefire config, I am using oracle jdk 8 > > >> > >> > > > >> > >> > <plugin> > > >> > >> > <artifactId>maven-surefire-plugin</artifactId> > > >> > >> > <version>2.20</version> > > >> > >> > <configuration> > > >> > >> > <forkCount>1</forkCount> > > >> > >> > <reuseForks>false</reuseForks> > > >> > >> > > > >> > >> > <forkedProcessTimeoutInSeconds>300</ > > forkedProcessTimeoutInSeconds> > > >> > >> > <argLine>-Xmx2G > > >> > >> > -Djava.io.tmpdir=${basedir}/target</argLine> > > >> > >> > </configuration> > > >> > >> > </plugin> > > >> > >> > > > >> > >> > -- Enrico > > >> > >> > > > >> > >> > > > >> > >> > > > >> > >> > > Sijie > > >> > >> > > > > >> > >> > > On Jul 10, 2017 1:17 AM, "Sijie Guo" <guosi...@gmail.com> > > wrote: > > >> > >> > > > > >> > >> > > > I am not sure if there is any default values changed for > > >> journal > > >> > >> > > settings. > > >> > >> > > > I would suggest you testing by setting specifically the > > journal > > >> > >> > settings. > > >> > >> > > > > > >> > >> > > > Also if you can share your benchmark, that would be good > for > > >> other > > >> > >> > people > > >> > >> > > > to verify. > > >> > >> > > > > > >> > >> > > > Sijie > > >> > >> > > > > > >> > >> > > > On Jul 10, 2017 12:32 AM, "Enrico Olivelli" < > > >> eolive...@gmail.com> > > >> > >> > wrote: > > >> > >> > > > > > >> > >> > > >> 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.setZkServ > > >> ers(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 > > >> > >> > > >> > > >> > >> > > > > > >> > >> > > > > >> > >> > > > >> > >> > > >> > >> > > >> > >> > > >> > >> -- > > >> > >> Jvrao > > >> > >> --- > > >> > >> First they ignore you, then they laugh at you, then they fight > you, > > >> then > > >> > >> you win. - Mahatma Gandhi > > >> > >> > > >> > > -- > > >> > > > > >> > > > > >> > > -- Enrico Olivelli > > >> > > > > >> > > > >> > > > > > > > > > > > > -- > Jvrao > --- > First they ignore you, then they laugh at you, then they fight you, then > you win. - Mahatma Gandhi >