You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by "hangc0276 (via GitHub)" <gi...@apache.org> on 2023/03/07 16:02:30 UTC

[GitHub] [bookkeeper] hangc0276 opened a new pull request, #3846: Streamline batch add request

hangc0276 opened a new pull request, #3846:
URL: https://github.com/apache/bookkeeper/pull/3846

   ### Motivation
   The bookie server process add-entry requests pipeline:
   - Get one request from the Netty socket channel
   - Choose one thread to process the written request
   - Write the entry into the target ledger entry logger's write cache(memory table)
   - Put the entry into the journal's pending queue
   - Journal thread takes the entry from the pending queue and writes it into PageCache/Journal Disk
   - Write the callback response to the Netty buffer and flush to the bookie client side.
   
   For every add entry request, the bookie server needs to go through the above steps one by one. It will introduce a lot of thread context switch. 
   
   We can batch the add requests according to the Netty socket channel, and write a batch of entries into the ledger entry logger and journal disk.
   
   ### Modifications
   The PR will change the add requests processing pipeline into the following steps.
   - Get a batch of add-entry requests from the socket channel until the socket channel is empty or reached the max capacity (default is 10_000)
   - Choose one thread to process the batch of add-entry requests.
   - Write the entries into the target ledger entry logger's write cache one by one
   - Put the batch of entries into the journal's pending queue
   - Journal thread drain a batch of entries from the pending queue and write them into PageCache/Journal disk
   - Write the callback response to the Netty buffer and flush to the bookie client side.
   
   With this change, we can save a lot of thread context switch.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3846: Streamline batch add request

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1134923932


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,143 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount;
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        startTimeNanos = -1L;
+    }
+
+    public static WriteBatchEntryProcessor create(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                                                  BookieRequestProcessor requestProcessor) {
+        WriteBatchEntryProcessor wbep = RECYCLER.get();
+        wbep.init(requests, requestHandler, requestProcessor);
+        requestProcessor.onAddRequestStart(requestHandler.ctx().channel(), requests.size());

Review Comment:
   I add the requests.size() instead of 1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] zymap commented on a diff in pull request #3846: Streamline batch add request

Posted by "zymap (via GitHub)" <gi...@apache.org>.
zymap commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1132115816


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/processor/RequestProcessor.java:
##########
@@ -47,4 +49,11 @@ public interface RequestProcessor extends AutoCloseable {
      * Flush any pending response staged on all the client connections.
      */
     void flushPendingResponses();
+
+    /**
+     * Process a list of ParsedAddRequests.
+     * @param r
+     * @param channel
+     */
+    void processAddRequest(List<BookieProtocol.ParsedAddRequest> r, BookieRequestHandler channel);

Review Comment:
   Looks like `void processRequest(Object r, BookieRequestHandler channel);` is the same as the new one. Because the argument is an object, we can reuse that method



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java:
##########
@@ -1087,6 +1093,114 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Obj
         }
     }
 
+    public void addEntryList(List<ParsedAddRequest> requests, boolean ackBeforeSync,
+                         WriteCallback cb, Object ctx, RequestStats requestStats) throws InterruptedException {
+        long requestNans = MathUtils.nowInNano();
+        boolean hasFailedRequests = false;
+        Map<Pair<Long, byte[]>, LedgerDescriptor> handleMap = new HashMap<>();
+        ListIterator<ParsedAddRequest> iter = requests.listIterator();
+        while (iter.hasNext()) {
+            ParsedAddRequest request = iter.next();
+            int rc = BookieProtocol.EOK;
+            try {
+                Pair<Long, byte[]> ledgerIdMasterKey = Pair.of(request.getLedgerId(), request.getMasterKey());
+                LedgerDescriptor handle = handleMap.get(ledgerIdMasterKey);
+                if (handle == null) {
+                    handle = getLedgerForEntry(request.getData(), request.getMasterKey());
+                    handleMap.put(ledgerIdMasterKey, handle);
+                }
+
+                synchronized (handle) {
+                    if (handle.isFenced()) {
+                        throw BookieException.create(BookieException.Code.LedgerFencedException);
+                    }
+
+                    addEntryInternalWithoutJournal(handle, request.getData(), ackBeforeSync,
+                        cb, ctx, request.getMasterKey());
+                }
+            } catch (BookieException.OperationRejectedException e) {
+                requestStats.getAddEntryRejectedCounter().inc();
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Operation rejected while writing {} ", request, e);
+                }
+                rc = BookieProtocol.ETOOMANYREQUESTS;
+            } catch (IOException e) {
+                LOG.error("Error writing {}", request, e);
+                rc = BookieProtocol.EIO;
+            } catch (BookieException.LedgerFencedException lfe) {
+                LOG.error("Attempt to write to fenced ledger ", lfe);
+                rc = BookieProtocol.EFENCED;
+            } catch (BookieException e) {
+                LOG.error("Unauthorized access to ledger {}", request.getLedgerId(), e);
+                rc = BookieProtocol.EUA;
+            } catch (Throwable t) {
+                LOG.error("Unexpected exception while writing {}@{} : {} ",
+                    request.getLedgerId(), request.getEntryId(), t.getMessage(), t);
+                rc = BookieProtocol.EBADREQ;
+            }
+
+            if (rc != BookieProtocol.EOK) {
+                hasFailedRequests = true;
+                requestStats.getAddEntryStats()
+                    .registerFailedEvent(MathUtils.elapsedNanos(requestNans), TimeUnit.NANOSECONDS);
+                cb.writeComplete(rc, request.getLedgerId(), request.getEntryId(), null, ctx);
+                iter.remove();
+                request.release();
+                request.recycle();
+            }
+        }
+        handleMap.clear();
+
+        if (hasFailedRequests && requestProcessor != null) {
+            requestProcessor.flushPendingResponses();
+        }
+
+        if (writeDataToJournal && !requests.isEmpty()) {
+            List<ByteBuf> entries = requests.stream()
+                .map(ParsedAddRequest::getData).collect(Collectors.toList());
+            getJournal(requests.get(0).getLedgerId()).logAddEntry(entries, ackBeforeSync, cb, ctx);
+
+            requests.forEach(t -> {
+                t.release();
+                t.recycle();
+            });
+        }
+    }
+
+    private void addEntryInternalWithoutJournal(LedgerDescriptor handle, ByteBuf entry,

Review Comment:
   We can easily add a flag at the original logic to avoid unnecessary duplicated code.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java:
##########
@@ -885,6 +888,25 @@ public void logAddEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb,
         logAddEntry(ledgerId, entryId, entry, ackBeforeSync, cb, ctx);
     }
 
+    public void logAddEntry(List<ByteBuf> entries, boolean ackBeforeSync, WriteCallback cb, Object ctx)
+        throws InterruptedException {
+        AtomicLong reserveMemory = new AtomicLong();
+        QueueEntry[] queueEntries = new QueueEntry[entries.size()];
+        for (int i = 0; i < entries.size(); ++i) {
+            ByteBuf entry = entries.get(i);
+            long ledgerId = entry.getLong(entry.readerIndex());
+            long entryId = entry.getLong(entry.readerIndex() + 8);
+            entry.retain();
+            reserveMemory.addAndGet(entry.readableBytes());
+            queueEntries[i] = QueueEntry.create(entry, ackBeforeSync, ledgerId, entryId, cb, ctx,
+                MathUtils.nowInNano(), journalStats.getJournalAddEntryStats(), callbackTime);
+        }
+
+        memoryLimitController.releaseMemory(reserveMemory.get());

Review Comment:
   reserveMemory?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java:
##########
@@ -229,6 +230,16 @@ protected void onAddRequestFinish() {
         }
     }
 
+    protected void onAddRequestFinishWithoutUnTrack() {
+        if (addsSemaphore != null) {
+            addsSemaphore.release();
+        }
+    }
+
+    protected void onAddRequestUnTrack() {
+        requestStats.untrackAddRequest();
+    }

Review Comment:
   You need to track N add requests at onAddRequestStart, and untrack it when onAddRequestFinish.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java:
##########
@@ -87,7 +97,32 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
             ctx.fireChannelRead(msg);
             return;
         }
-        requestProcessor.processRequest(msg, this);
+
+        if (msg instanceof BookieProtocol.ParsedAddRequest
+            && ADDENTRY == ((BookieProtocol.ParsedAddRequest) msg).getOpCode()
+            && !((BookieProtocol.ParsedAddRequest) msg).isHighPriority()
+            && ((BookieProtocol.ParsedAddRequest) msg).getProtocolVersion() == BookieProtocol.CURRENT_PROTOCOL_VERSION
+            && !((BookieProtocol.ParsedAddRequest) msg).isRecoveryAdd()) {
+            msgs.put((BookieProtocol.ParsedAddRequest) msg);

Review Comment:
   Put is a blocking operation. We should check the size first and then put the message into the queue.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java:
##########
@@ -87,7 +97,32 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
             ctx.fireChannelRead(msg);
             return;
         }
-        requestProcessor.processRequest(msg, this);
+
+        if (msg instanceof BookieProtocol.ParsedAddRequest
+            && ADDENTRY == ((BookieProtocol.ParsedAddRequest) msg).getOpCode()
+            && !((BookieProtocol.ParsedAddRequest) msg).isHighPriority()
+            && ((BookieProtocol.ParsedAddRequest) msg).getProtocolVersion() == BookieProtocol.CURRENT_PROTOCOL_VERSION
+            && !((BookieProtocol.ParsedAddRequest) msg).isRecoveryAdd()) {
+            msgs.put((BookieProtocol.ParsedAddRequest) msg);
+            if (msgs.size() >= maxCapacity) {
+                int count = msgs.size();
+                List<BookieProtocol.ParsedAddRequest> c = new ArrayList<>(count);
+                msgs.drainTo(c, count);
+                requestProcessor.processAddRequest(c, this);
+            }

Review Comment:
   ```suggestion
               if (!msgs.offer(msg)) {
                   ctx.fireChannelReadComplete();
               }
               msgs.put(msg); 
   ```



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java:
##########
@@ -729,4 +740,28 @@ public boolean isBlacklisted(Channel channel) {
     public void handleNonWritableChannel(Channel channel) {
         onResponseTimeout.accept(channel);
     }
+
+    @Override
+    public void processAddRequest(List<BookieProtocol.ParsedAddRequest> msgs, BookieRequestHandler requestHandler) {
+        WriteBatchEntryProcessor write = WriteBatchEntryProcessor.create(msgs, requestHandler, this);

Review Comment:
   You can use[ the original method](https://github.com/apache/bookkeeper/pull/3846/files#diff-380ca68ed5ce21fb226c59b8f9c9bd7c7be70da37091ea2c55db89cc4125e578R318) and when the Object msg is `List<BookieProtocol.ParsedAddRequest>` then call here. They do the same thing just in different ways, so let's reuse the existing method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3846: Streamline batch add request

Posted by "eolivelli (via GitHub)" <gi...@apache.org>.
eolivelli commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1136657665


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java:
##########
@@ -1087,6 +1096,80 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Obj
         }
     }
 
+    public void addEntryList(List<ParsedAddRequest> requests, boolean ackBeforeSync,
+                         WriteCallback cb, Object ctx, RequestStats requestStats) throws InterruptedException {
+        long requestNans = MathUtils.nowInNano();
+        boolean hasFailedRequests = false;
+        Map<Pair<Long, byte[]>, LedgerDescriptor> handleMap = new HashMap<>();
+        ListIterator<ParsedAddRequest> iter = requests.listIterator();
+        while (iter.hasNext()) {
+            ParsedAddRequest request = iter.next();
+            int rc = BookieProtocol.EOK;
+            try {
+                Pair<Long, byte[]> ledgerIdMasterKey = Pair.of(request.getLedgerId(), request.getMasterKey());
+                LedgerDescriptor handle = handleMap.get(ledgerIdMasterKey);
+                if (handle == null) {
+                    handle = getLedgerForEntry(request.getData(), request.getMasterKey());
+                    handleMap.put(ledgerIdMasterKey, handle);
+                }
+
+                synchronized (handle) {
+                    if (handle.isFenced()) {
+                        throw BookieException.create(BookieException.Code.LedgerFencedException);
+                    }
+
+                    addEntryInternal(handle, request.getData(), ackBeforeSync,
+                        cb, ctx, request.getMasterKey(), false);
+                }
+            } catch (BookieException.OperationRejectedException e) {
+                requestStats.getAddEntryRejectedCounter().inc();
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Operation rejected while writing {} ", request, e);
+                }
+                rc = BookieProtocol.ETOOMANYREQUESTS;
+            } catch (IOException e) {
+                LOG.error("Error writing {}", request, e);
+                rc = BookieProtocol.EIO;
+            } catch (BookieException.LedgerFencedException lfe) {
+                LOG.error("Attempt to write to fenced ledger ", lfe);
+                rc = BookieProtocol.EFENCED;
+            } catch (BookieException e) {
+                LOG.error("Unauthorized access to ledger {}", request.getLedgerId(), e);
+                rc = BookieProtocol.EUA;
+            } catch (Throwable t) {
+                LOG.error("Unexpected exception while writing {}@{} : {} ",
+                    request.getLedgerId(), request.getEntryId(), t.getMessage(), t);
+                rc = BookieProtocol.EBADREQ;
+            }
+
+            if (rc != BookieProtocol.EOK) {
+                hasFailedRequests = true;
+                requestStats.getAddEntryStats()
+                    .registerFailedEvent(MathUtils.elapsedNanos(requestNans), TimeUnit.NANOSECONDS);
+                cb.writeComplete(rc, request.getLedgerId(), request.getEntryId(), null, ctx);
+                iter.remove();
+                request.release();
+                request.recycle();
+            }
+        }
+        handleMap.clear();
+
+        if (hasFailedRequests && requestProcessor != null) {
+            requestProcessor.flushPendingResponses();
+        }
+
+        if (writeDataToJournal && !requests.isEmpty()) {
+            List<ByteBuf> entries = requests.stream()
+                .map(ParsedAddRequest::getData).collect(Collectors.toList());
+            getJournal(requests.get(0).getLedgerId()).logAddEntry(entries, ackBeforeSync, cb, ctx);
+        }
+
+        requests.forEach(t -> {

Review Comment:
   put this into a "finally" block ?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java:
##########
@@ -885,6 +887,26 @@ public void logAddEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb,
         logAddEntry(ledgerId, entryId, entry, ackBeforeSync, cb, ctx);
     }
 
+    public void logAddEntry(List<ByteBuf> entries, boolean ackBeforeSync, WriteCallback cb, Object ctx)

Review Comment:
   nit: logAddEntries



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java:
##########
@@ -1087,6 +1096,80 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Obj
         }
     }
 
+    public void addEntryList(List<ParsedAddRequest> requests, boolean ackBeforeSync,
+                         WriteCallback cb, Object ctx, RequestStats requestStats) throws InterruptedException {
+        long requestNans = MathUtils.nowInNano();
+        boolean hasFailedRequests = false;
+        Map<Pair<Long, byte[]>, LedgerDescriptor> handleMap = new HashMap<>();
+        ListIterator<ParsedAddRequest> iter = requests.listIterator();
+        while (iter.hasNext()) {
+            ParsedAddRequest request = iter.next();
+            int rc = BookieProtocol.EOK;
+            try {
+                Pair<Long, byte[]> ledgerIdMasterKey = Pair.of(request.getLedgerId(), request.getMasterKey());
+                LedgerDescriptor handle = handleMap.get(ledgerIdMasterKey);
+                if (handle == null) {
+                    handle = getLedgerForEntry(request.getData(), request.getMasterKey());
+                    handleMap.put(ledgerIdMasterKey, handle);
+                }
+
+                synchronized (handle) {
+                    if (handle.isFenced()) {
+                        throw BookieException.create(BookieException.Code.LedgerFencedException);
+                    }
+
+                    addEntryInternal(handle, request.getData(), ackBeforeSync,
+                        cb, ctx, request.getMasterKey(), false);
+                }
+            } catch (BookieException.OperationRejectedException e) {
+                requestStats.getAddEntryRejectedCounter().inc();
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Operation rejected while writing {} ", request, e);
+                }
+                rc = BookieProtocol.ETOOMANYREQUESTS;
+            } catch (IOException e) {
+                LOG.error("Error writing {}", request, e);
+                rc = BookieProtocol.EIO;
+            } catch (BookieException.LedgerFencedException lfe) {
+                LOG.error("Attempt to write to fenced ledger ", lfe);
+                rc = BookieProtocol.EFENCED;
+            } catch (BookieException e) {
+                LOG.error("Unauthorized access to ledger {}", request.getLedgerId(), e);
+                rc = BookieProtocol.EUA;
+            } catch (Throwable t) {
+                LOG.error("Unexpected exception while writing {}@{} : {} ",
+                    request.getLedgerId(), request.getEntryId(), t.getMessage(), t);
+                rc = BookieProtocol.EBADREQ;
+            }
+
+            if (rc != BookieProtocol.EOK) {
+                hasFailedRequests = true;
+                requestStats.getAddEntryStats()
+                    .registerFailedEvent(MathUtils.elapsedNanos(requestNans), TimeUnit.NANOSECONDS);
+                cb.writeComplete(rc, request.getLedgerId(), request.getEntryId(), null, ctx);
+                iter.remove();
+                request.release();
+                request.recycle();
+            }
+        }
+        handleMap.clear();
+
+        if (hasFailedRequests && requestProcessor != null) {
+            requestProcessor.flushPendingResponses();
+        }
+
+        if (writeDataToJournal && !requests.isEmpty()) {
+            List<ByteBuf> entries = requests.stream()
+                .map(ParsedAddRequest::getData).collect(Collectors.toList());
+            getJournal(requests.get(0).getLedgerId()).logAddEntry(entries, ackBeforeSync, cb, ctx);

Review Comment:
   here we are assuming that all the entries are for the same ledger.
   Can we add some assertions ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3846: Streamline batch add request

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1138106154


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java:
##########
@@ -87,7 +97,33 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
             ctx.fireChannelRead(msg);
             return;
         }
-        requestProcessor.processRequest(msg, this);
+
+        if (msg instanceof BookieProtocol.ParsedAddRequest
+            && ADDENTRY == ((BookieProtocol.ParsedAddRequest) msg).getOpCode()
+            && !((BookieProtocol.ParsedAddRequest) msg).isHighPriority()
+            && ((BookieProtocol.ParsedAddRequest) msg).getProtocolVersion() == BookieProtocol.CURRENT_PROTOCOL_VERSION
+            && !((BookieProtocol.ParsedAddRequest) msg).isRecoveryAdd()) {
+            BookieProtocol.ParsedAddRequest request = (BookieProtocol.ParsedAddRequest) msg;
+            if (!msgs.offer(request)) {
+                channelReadComplete(ctx);
+                msgs.put(request);
+            }
+        } else {
+            requestProcessor.processRequest(msg, this);
+        }
+    }
+
+    @Override
+    public void channelReadComplete(ChannelHandlerContext ctx) {
+        if (!msgs.isEmpty()) {
+            int count = msgs.size();
+            log.info("[hangc] count: {}", count);

Review Comment:
   removed it.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,143 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount = new AtomicInteger(0);
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        requestCount.set(0);
+        startTimeNanos = -1L;
+    }
+
+    public static WriteBatchEntryProcessor create(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                                                  BookieRequestProcessor requestProcessor) {
+        WriteBatchEntryProcessor wbep = RECYCLER.get();
+        wbep.init(requests, requestHandler, requestProcessor);
+        requestProcessor.onAddRequestStart(requestHandler.ctx().channel(), requests.size());
+        return wbep;
+    }
+
+    protected void init(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                      BookieRequestProcessor requestProcessor) {
+        this.requests = requests;
+        this.requestHandler = requestHandler;
+        this.requestProcessor = requestProcessor;
+        this.enqueueNanos = MathUtils.nowInNano();
+        this.requestCount.set(requests.size());
+    }
+
+    @Override
+    protected void processPacket() {
+
+    }
+
+    @Override
+    public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) {
+        if (BookieProtocol.EOK == rc) {
+            requestProcessor.getRequestStats().getAddEntryStats()
+                .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS);
+        } else {
+            requestProcessor.getRequestStats().getAddEntryStats()
+                .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS);
+        }
+
+        requestHandler.prepareSendResponseV2(rc, BookieProtocol.CURRENT_PROTOCOL_VERSION, ADDENTRY, ledgerId, entryId);
+        requestProcessor.onAddRequestFinish();
+
+        if (requestCount.decrementAndGet() == 0) {
+            recycle();
+        }
+    }
+
+    @Override
+    public void run() {
+        if (requestProcessor.getBookie().isReadOnly()) {
+                log.warn("BookieServer is running in readOnly mode, so rejecting the request from the client!");
+                for (ParsedAddRequest r : requests) {
+                    writeComplete(BookieProtocol.EREADONLY, r.getLedgerId(), r.getEntryId(), null,
+                        requestHandler.ctx());
+                    r.release();
+                    r.recycle();
+                }
+                return;
+        }
+

Review Comment:
   Good point, added it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3846: Streamline batch add request

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1137992803


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java:
##########
@@ -1087,6 +1096,80 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Obj
         }
     }
 
+    public void addEntryList(List<ParsedAddRequest> requests, boolean ackBeforeSync,
+                         WriteCallback cb, Object ctx, RequestStats requestStats) throws InterruptedException {
+        long requestNans = MathUtils.nowInNano();
+        boolean hasFailedRequests = false;
+        Map<Pair<Long, byte[]>, LedgerDescriptor> handleMap = new HashMap<>();
+        ListIterator<ParsedAddRequest> iter = requests.listIterator();
+        while (iter.hasNext()) {
+            ParsedAddRequest request = iter.next();
+            int rc = BookieProtocol.EOK;
+            try {
+                Pair<Long, byte[]> ledgerIdMasterKey = Pair.of(request.getLedgerId(), request.getMasterKey());
+                LedgerDescriptor handle = handleMap.get(ledgerIdMasterKey);
+                if (handle == null) {
+                    handle = getLedgerForEntry(request.getData(), request.getMasterKey());
+                    handleMap.put(ledgerIdMasterKey, handle);
+                }
+
+                synchronized (handle) {
+                    if (handle.isFenced()) {
+                        throw BookieException.create(BookieException.Code.LedgerFencedException);
+                    }
+
+                    addEntryInternal(handle, request.getData(), ackBeforeSync,
+                        cb, ctx, request.getMasterKey(), false);
+                }
+            } catch (BookieException.OperationRejectedException e) {
+                requestStats.getAddEntryRejectedCounter().inc();
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Operation rejected while writing {} ", request, e);
+                }
+                rc = BookieProtocol.ETOOMANYREQUESTS;
+            } catch (IOException e) {
+                LOG.error("Error writing {}", request, e);
+                rc = BookieProtocol.EIO;
+            } catch (BookieException.LedgerFencedException lfe) {
+                LOG.error("Attempt to write to fenced ledger ", lfe);
+                rc = BookieProtocol.EFENCED;
+            } catch (BookieException e) {
+                LOG.error("Unauthorized access to ledger {}", request.getLedgerId(), e);
+                rc = BookieProtocol.EUA;
+            } catch (Throwable t) {
+                LOG.error("Unexpected exception while writing {}@{} : {} ",
+                    request.getLedgerId(), request.getEntryId(), t.getMessage(), t);
+                rc = BookieProtocol.EBADREQ;
+            }
+
+            if (rc != BookieProtocol.EOK) {
+                hasFailedRequests = true;
+                requestStats.getAddEntryStats()
+                    .registerFailedEvent(MathUtils.elapsedNanos(requestNans), TimeUnit.NANOSECONDS);
+                cb.writeComplete(rc, request.getLedgerId(), request.getEntryId(), null, ctx);
+                iter.remove();
+                request.release();
+                request.recycle();
+            }
+        }
+        handleMap.clear();
+
+        if (hasFailedRequests && requestProcessor != null) {
+            requestProcessor.flushPendingResponses();
+        }
+
+        if (writeDataToJournal && !requests.isEmpty()) {
+            List<ByteBuf> entries = requests.stream()
+                .map(ParsedAddRequest::getData).collect(Collectors.toList());
+            getJournal(requests.get(0).getLedgerId()).logAddEntry(entries, ackBeforeSync, cb, ctx);

Review Comment:
   We don't assume all the entries belong to the same ledger. When we write entries in the journal, we don't need to write all the ledger's data into the same journal directory. The goal of `getJournal(requests.get(0).getLedgerId())` is to balance the write throughput between journals.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] horizonzy commented on a diff in pull request #3846: Streamline batch add request

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1135029504


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,144 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount;
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        requestCount = null;

Review Comment:
   Here we need not set `requestCount = null`, we can set requestCount.set(request.size()) in the init method to override the value.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] wenbingshen commented on a diff in pull request #3846: Streamline batch add request

Posted by "wenbingshen (via GitHub)" <gi...@apache.org>.
wenbingshen commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1138037210


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java:
##########
@@ -34,20 +40,24 @@
  */
 @Slf4j
 public class BookieRequestHandler extends ChannelInboundHandlerAdapter {
-
+    private static final int DEFAULT_CAPACITY = 1_000;
     static final Object EVENT_FLUSH_ALL_PENDING_RESPONSES = new Object();
 
     private final RequestProcessor requestProcessor;
     private final ChannelGroup allChannels;
 
     private ChannelHandlerContext ctx;
+    private final BlockingQueue<BookieProtocol.ParsedAddRequest> msgs;
 
     private ByteBuf pendingSendResponses = null;
     private int maxPendingResponsesSize;
 
     BookieRequestHandler(ServerConfiguration conf, RequestProcessor processor, ChannelGroup allChannels) {
         this.requestProcessor = processor;
         this.allChannels = allChannels;
+
+        int maxCapacity = conf.getMaxAddsInProgressLimit() > 0 ? conf.getMaxAddsInProgressLimit() : DEFAULT_CAPACITY;
+        this.msgs = new ArrayBlockingQueue<>(maxCapacity);

Review Comment:
   I got it. Thanks!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,143 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount = new AtomicInteger(0);
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        requestCount.set(0);
+        startTimeNanos = -1L;
+    }
+
+    public static WriteBatchEntryProcessor create(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                                                  BookieRequestProcessor requestProcessor) {
+        WriteBatchEntryProcessor wbep = RECYCLER.get();
+        wbep.init(requests, requestHandler, requestProcessor);
+        requestProcessor.onAddRequestStart(requestHandler.ctx().channel(), requests.size());
+        return wbep;
+    }
+
+    protected void init(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                      BookieRequestProcessor requestProcessor) {
+        this.requests = requests;
+        this.requestHandler = requestHandler;
+        this.requestProcessor = requestProcessor;
+        this.enqueueNanos = MathUtils.nowInNano();
+        this.requestCount.set(requests.size());
+    }
+
+    @Override
+    protected void processPacket() {
+
+    }
+
+    @Override
+    public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) {
+        if (BookieProtocol.EOK == rc) {
+            requestProcessor.getRequestStats().getAddEntryStats()
+                .registerSuccessfulEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS);
+        } else {
+            requestProcessor.getRequestStats().getAddEntryStats()
+                .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS);
+        }
+
+        requestHandler.prepareSendResponseV2(rc, BookieProtocol.CURRENT_PROTOCOL_VERSION, ADDENTRY, ledgerId, entryId);
+        requestProcessor.onAddRequestFinish();
+
+        if (requestCount.decrementAndGet() == 0) {
+            recycle();
+        }
+    }
+
+    @Override
+    public void run() {
+        if (requestProcessor.getBookie().isReadOnly()) {
+                log.warn("BookieServer is running in readOnly mode, so rejecting the request from the client!");
+                for (ParsedAddRequest r : requests) {
+                    writeComplete(BookieProtocol.EREADONLY, r.getLedgerId(), r.getEntryId(), null,
+                        requestHandler.ctx());
+                    r.release();
+                    r.recycle();
+                }
+                return;
+        }
+

Review Comment:
   When batch add, do we need to update the `writeThreadQueuedLatency` metric? like: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBase.java#L176-L184



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] Shoothzj commented on pull request #3846: Streamline batch add request

Posted by "Shoothzj (via GitHub)" <gi...@apache.org>.
Shoothzj commented on PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#issuecomment-1463438974

   Can you share some relevant performance testing data for this PR?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] horizonzy commented on a diff in pull request #3846: Streamline batch add request

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1138020842


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java:
##########
@@ -87,7 +97,33 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
             ctx.fireChannelRead(msg);
             return;
         }
-        requestProcessor.processRequest(msg, this);
+
+        if (msg instanceof BookieProtocol.ParsedAddRequest
+            && ADDENTRY == ((BookieProtocol.ParsedAddRequest) msg).getOpCode()
+            && !((BookieProtocol.ParsedAddRequest) msg).isHighPriority()
+            && ((BookieProtocol.ParsedAddRequest) msg).getProtocolVersion() == BookieProtocol.CURRENT_PROTOCOL_VERSION
+            && !((BookieProtocol.ParsedAddRequest) msg).isRecoveryAdd()) {
+            BookieProtocol.ParsedAddRequest request = (BookieProtocol.ParsedAddRequest) msg;
+            if (!msgs.offer(request)) {
+                channelReadComplete(ctx);
+                msgs.put(request);
+            }
+        } else {
+            requestProcessor.processRequest(msg, this);
+        }
+    }
+
+    @Override
+    public void channelReadComplete(ChannelHandlerContext ctx) {
+        if (!msgs.isEmpty()) {
+            int count = msgs.size();
+            log.info("[hangc] count: {}", count);

Review Comment:
   maybe it's a test log.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] horizonzy commented on a diff in pull request #3846: Streamline batch add request

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1147148509


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java:
##########
@@ -205,21 +206,21 @@ public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, Stat
         readsSemaphore = maxReads > 0 ? new Semaphore(maxReads, true) : null;
     }
 
-    protected void onAddRequestStart(Channel channel) {
+    protected void onAddRequestStart(Channel channel, int permits) {
         if (addsSemaphore != null) {
-            if (!addsSemaphore.tryAcquire()) {
+            if (!addsSemaphore.tryAcquire(permits)) {
                 final long throttlingStartTimeNanos = MathUtils.nowInNano();
                 channel.config().setAutoRead(false);
                 LOG.info("Too many add requests in progress, disabling autoread on channel {}", channel);
                 requestStats.blockAddRequest();
-                addsSemaphore.acquireUninterruptibly();
+                addsSemaphore.acquireUninterruptibly(permits);

Review Comment:
   +1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] XLzed commented on pull request #3846: Streamline batch add request

Posted by "XLzed (via GitHub)" <gi...@apache.org>.
XLzed commented on PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#issuecomment-1586077171

   Hello! I'm a beginner of bookkeeper. Could you share more details about the benchmark like bk configuration and hardware enviroment? I want to use this change and learn the optimization method within it. But when I tried to reproduce the result of your benchmark, the results looked a little odd, the optimized version of protocolV2 performed about as well as the unoptimized version of protocolV3. What's the difference between V2 and V3?And why this optimization supports V2 only?
   Here's my hardware enviroment and benchmark results, thank you!
   
   server cmd:OPTS='-XX:MaxDirectMemorySize=10G -Xms40G -Xmx40G' ./bin/bookkeeper localbookie 1
   client cmd:OPTS='-Dorg.apache.bookkeeper.conf.readsystemproperties=true -DuseV2WireProtocol=true' ./bookkeeper-benchmark/bin/benchmark writes -ensemble 1 -quorum 1 -ackQuorum 1 -ledgers 50 -throttle 10000
   
   ![image](https://github.com/apache/bookkeeper/assets/46588381/46899415-cd9b-4fb4-9326-06c9367f3804)
   
   Before Change(useV2WireProtocol=false)
   <html>
   <body>
   <!--StartFragment--><div data-sabo-line="true" style="margin: 0px; white-space: pre;">
   
   times | ops/sec | p95 latency | p99 latency
   -- | -- | -- | --
   1 | 212980 | 41.68268 | 44.85271
   2 | 202530 | 42.99525 | 46.63221
   3 | 212138 | 42.30779 | 45.08659
   4 | 213154 | 42.22976 | 44.76916
   5 | 198376 | 42.3406 | 47.08146
   Avg | 207835.6 | 42.31122 | 45.68443
   
   </div><div data-sabo-app-id="note.temu.team"></div><!--EndFragment-->
   </body>
   </html>
   
   Before Change (useV2WireProtocol=true)
   <html>
   <body>
   <!--StartFragment--><div data-sabo-line="true" style="margin: 0px; white-space: pre;">
   
   times | ops/sec | p95 latency | p99 latency
   -- | -- | -- | --
   1 | 195363 | 46.08798 | 49.72923
   2 | 198560 | 45.37565 | 48.83366
   3 | 198490 | 45.53701 | 49.00147
   4 | 202162 | 44.70965 | 48.1483
   5 | 197736 | 45.71856 | 49.23503
   Avg | 198462.2 | 45.48577 | 48.98954
   
   </div><div data-sabo-app-id="note.temu.team"></div><!--EndFragment-->
   </body>
   </html>
   
   After Change (useV2WireProtocol=true)
   <html>
   <body>
   <!--StartFragment--><div data-sabo-line="true" style="margin: 0px; white-space: pre;">
   
   times | ops/sec | p95 latency | p99 latency
   -- | -- | -- | --
   1 | 211978 | 41.49986 | 44.78894
   2 | 216729 | 41.85365 | 44.18242
   3 | 208047 | 42.2268 | 45.47032
   4 | 204107 | 41.99345 | 46.13609
   5 | 203803 | 42.82836 | 46.37437
   Avg | 208932.8 | 42.08043 | 45.39043
   
   </div><div data-sabo-app-id="note.temu.team"></div><!--EndFragment-->
   </body>
   </html>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3846: Streamline batch add request

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1135049146


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,143 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount;
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        startTimeNanos = -1L;
+    }
+
+    public static WriteBatchEntryProcessor create(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                                                  BookieRequestProcessor requestProcessor) {
+        WriteBatchEntryProcessor wbep = RECYCLER.get();
+        wbep.init(requests, requestHandler, requestProcessor);
+        requestProcessor.onAddRequestStart(requestHandler.ctx().channel(), requests.size());

Review Comment:
   got it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] codecov-commenter commented on pull request #3846: Streamline batch add request

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#issuecomment-1458486183

   # [Codecov](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3846](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9db1e1d) into [master](https://codecov.io/gh/apache/bookkeeper/commit/d6748f915d4801e90f001bc09d65918df85b305f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d6748f9) will **decrease** coverage by `10.77%`.
   > The diff coverage is `54.18%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #3846       +/-   ##
   =============================================
   - Coverage     60.40%   49.63%   -10.77%     
   + Complexity     5862     4828     -1034     
   =============================================
     Files           473      474        +1     
     Lines         40960    41133      +173     
     Branches       5240     5262       +22     
   =============================================
   - Hits          24740    20418     -4322     
   - Misses        14002    18680     +4678     
   + Partials       2218     2035      -183     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | bookie | `39.76% <53.07%> (-0.01%)` | :arrow_down: |
   | remaining | `29.45% <2.79%> (-0.16%)` | :arrow_down: |
   | replication | `?` | |
   | tls | `21.04% <54.18%> (+0.05%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...main/java/org/apache/bookkeeper/bookie/Bookie.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvYm9va2llL0Jvb2tpZS5qYXZh) | `75.00% <ø> (ø)` | |
   | [...va/org/apache/bookkeeper/proto/BookieProtocol.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvcHJvdG8vQm9va2llUHJvdG9jb2wuamF2YQ==) | `80.71% <ø> (-2.86%)` | :arrow_down: |
   | [...g/apache/bookkeeper/proto/WriteEntryProcessor.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvcHJvdG8vV3JpdGVFbnRyeVByb2Nlc3Nvci5qYXZh) | `0.00% <0.00%> (-69.02%)` | :arrow_down: |
   | [...pache/bookkeeper/proto/BookieRequestProcessor.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvcHJvdG8vQm9va2llUmVxdWVzdFByb2Nlc3Nvci5qYXZh) | `46.23% <22.22%> (-15.29%)` | :arrow_down: |
   | [.../java/org/apache/bookkeeper/bookie/BookieImpl.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvYm9va2llL0Jvb2tpZUltcGwuamF2YQ==) | `67.09% <46.77%> (-3.13%)` | :arrow_down: |
   | [.../apache/bookkeeper/proto/BookieRequestHandler.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvcHJvdG8vQm9va2llUmVxdWVzdEhhbmRsZXIuamF2YQ==) | `55.55% <50.00%> (-30.81%)` | :arrow_down: |
   | [...che/bookkeeper/proto/WriteBatchEntryProcessor.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvcHJvdG8vV3JpdGVCYXRjaEVudHJ5UHJvY2Vzc29yLmphdmE=) | `61.29% <61.29%> (ø)` | |
   | [...ain/java/org/apache/bookkeeper/bookie/Journal.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvYm9va2llL0pvdXJuYWwuamF2YQ==) | `78.01% <100.00%> (-1.66%)` | :arrow_down: |
   | [...a/org/apache/bookkeeper/proto/ResponseBuilder.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvcHJvdG8vUmVzcG9uc2VCdWlsZGVyLmphdmE=) | `33.33% <100.00%> (-39.40%)` | :arrow_down: |
   | [...apache/bookkeeper/proto/WriteEntryProcessorV3.java](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Ym9va2tlZXBlci1zZXJ2ZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvcHJvdG8vV3JpdGVFbnRyeVByb2Nlc3NvclYzLmphdmE=) | `64.21% <100.00%> (-10.53%)` | :arrow_down: |
   | ... and [179 more](https://codecov.io/gh/apache/bookkeeper/pull/3846?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] horizonzy commented on a diff in pull request #3846: Streamline batch add request

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1135026819


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,143 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount;
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        startTimeNanos = -1L;
+    }
+
+    public static WriteBatchEntryProcessor create(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                                                  BookieRequestProcessor requestProcessor) {
+        WriteBatchEntryProcessor wbep = RECYCLER.get();
+        wbep.init(requests, requestHandler, requestProcessor);
+        requestProcessor.onAddRequestStart(requestHandler.ctx().channel(), requests.size());

Review Comment:
   The `addsSemaphore` use requests.size().
   The `addsInProgress` use 1.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] horizonzy commented on a diff in pull request #3846: Streamline batch add request

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1147148509


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java:
##########
@@ -205,21 +206,21 @@ public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, Stat
         readsSemaphore = maxReads > 0 ? new Semaphore(maxReads, true) : null;
     }
 
-    protected void onAddRequestStart(Channel channel) {
+    protected void onAddRequestStart(Channel channel, int permits) {
         if (addsSemaphore != null) {
-            if (!addsSemaphore.tryAcquire()) {
+            if (!addsSemaphore.tryAcquire(permits)) {
                 final long throttlingStartTimeNanos = MathUtils.nowInNano();
                 channel.config().setAutoRead(false);
                 LOG.info("Too many add requests in progress, disabling autoread on channel {}", channel);
                 requestStats.blockAddRequest();
-                addsSemaphore.acquireUninterruptibly();
+                addsSemaphore.acquireUninterruptibly(permits);

Review Comment:
   +1, we should ensure the batch size is less than addsSemaphore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] wenbingshen commented on a diff in pull request #3846: Streamline batch add request

Posted by "wenbingshen (via GitHub)" <gi...@apache.org>.
wenbingshen commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1137181222


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java:
##########
@@ -34,20 +40,24 @@
  */
 @Slf4j
 public class BookieRequestHandler extends ChannelInboundHandlerAdapter {
-
+    private static final int DEFAULT_CAPACITY = 1_000;
     static final Object EVENT_FLUSH_ALL_PENDING_RESPONSES = new Object();
 
     private final RequestProcessor requestProcessor;
     private final ChannelGroup allChannels;
 
     private ChannelHandlerContext ctx;
+    private final BlockingQueue<BookieProtocol.ParsedAddRequest> msgs;
 
     private ByteBuf pendingSendResponses = null;
     private int maxPendingResponsesSize;
 
     BookieRequestHandler(ServerConfiguration conf, RequestProcessor processor, ChannelGroup allChannels) {
         this.requestProcessor = processor;
         this.allChannels = allChannels;
+
+        int maxCapacity = conf.getMaxAddsInProgressLimit() > 0 ? conf.getMaxAddsInProgressLimit() : DEFAULT_CAPACITY;
+        this.msgs = new ArrayBlockingQueue<>(maxCapacity);

Review Comment:
   I'm not sure if it's going to happen, but I wanted to ask this question: Do we need to add a byte size limit? How to prevent the channel from reading a large amount of data from the socket and causing the OOM of the direct memory?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3846: Streamline batch add request

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1133741341


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java:
##########
@@ -229,6 +230,16 @@ protected void onAddRequestFinish() {
         }
     }
 
+    protected void onAddRequestFinishWithoutUnTrack() {
+        if (addsSemaphore != null) {
+            addsSemaphore.release();
+        }
+    }
+
+    protected void onAddRequestUnTrack() {
+        requestStats.untrackAddRequest();
+    }

Review Comment:
   We batched the add requests, but each request callback one by one. So I tracked N add requests at onAddRequestStart, but I should not call the onAddRequestFinish after all the entries callback reached. We need to release the addsSemaphore when one callback is reached and un-track the add requests after all the entries callback reached.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] horizonzy commented on a diff in pull request #3846: Streamline batch add request

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1134833855


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,143 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount;
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        startTimeNanos = -1L;
+    }
+
+    public static WriteBatchEntryProcessor create(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                                                  BookieRequestProcessor requestProcessor) {
+        WriteBatchEntryProcessor wbep = RECYCLER.get();
+        wbep.init(requests, requestHandler, requestProcessor);
+        requestProcessor.onAddRequestStart(requestHandler.ctx().channel(), requests.size());
+        return wbep;
+    }
+
+    protected void init(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                      BookieRequestProcessor requestProcessor) {
+        this.requests = requests;
+        this.requestHandler = requestHandler;
+        this.requestProcessor = requestProcessor;
+        this.enqueueNanos = MathUtils.nowInNano();
+        this.requestCount = new AtomicInteger(requests.size());

Review Comment:
   We can use requestCount.set(requests.size()) to reuse the AtomicInteger Object.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java:
##########
@@ -885,6 +887,25 @@ public void logAddEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb,
         logAddEntry(ledgerId, entryId, entry, ackBeforeSync, cb, ctx);
     }
 
+    public void logAddEntry(List<ByteBuf> entries, boolean ackBeforeSync, WriteCallback cb, Object ctx)
+        throws InterruptedException {
+        long reserveMemory = 0;
+        QueueEntry[] queueEntries = new QueueEntry[entries.size()];
+        for (int i = 0; i < entries.size(); ++i) {
+            ByteBuf entry = entries.get(i);
+            long ledgerId = entry.getLong(entry.readerIndex());
+            long entryId = entry.getLong(entry.readerIndex() + 8);
+            entry.retain();
+            reserveMemory += entry.readableBytes();
+            queueEntries[i] = QueueEntry.create(entry, ackBeforeSync, ledgerId, entryId, cb, ctx,
+                MathUtils.nowInNano(), journalStats.getJournalAddEntryStats(), callbackTime);
+        }
+
+        memoryLimitController.releaseMemory(reserveMemory);

Review Comment:
   -> memoryLimitController.reserveMemory(reserveMemory);



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java:
##########
@@ -1087,6 +1096,80 @@ public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, Obj
         }
     }
 
+    public void addEntryList(List<ParsedAddRequest> requests, boolean ackBeforeSync,
+                         WriteCallback cb, Object ctx, RequestStats requestStats) throws InterruptedException {
+        long requestNans = MathUtils.nowInNano();
+        boolean hasFailedRequests = false;
+        Map<Pair<Long, byte[]>, LedgerDescriptor> handleMap = new HashMap<>();
+        ListIterator<ParsedAddRequest> iter = requests.listIterator();
+        while (iter.hasNext()) {
+            ParsedAddRequest request = iter.next();
+            int rc = BookieProtocol.EOK;
+            try {
+                Pair<Long, byte[]> ledgerIdMasterKey = Pair.of(request.getLedgerId(), request.getMasterKey());
+                LedgerDescriptor handle = handleMap.get(ledgerIdMasterKey);
+                if (handle == null) {
+                    handle = getLedgerForEntry(request.getData(), request.getMasterKey());
+                    handleMap.put(ledgerIdMasterKey, handle);
+                }
+
+                synchronized (handle) {
+                    if (handle.isFenced()) {
+                        throw BookieException.create(BookieException.Code.LedgerFencedException);
+                    }
+
+                    addEntryInternal(handle, request.getData(), ackBeforeSync,
+                        cb, ctx, request.getMasterKey(), false);
+                }
+            } catch (BookieException.OperationRejectedException e) {
+                requestStats.getAddEntryRejectedCounter().inc();
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Operation rejected while writing {} ", request, e);
+                }
+                rc = BookieProtocol.ETOOMANYREQUESTS;
+            } catch (IOException e) {
+                LOG.error("Error writing {}", request, e);
+                rc = BookieProtocol.EIO;
+            } catch (BookieException.LedgerFencedException lfe) {
+                LOG.error("Attempt to write to fenced ledger ", lfe);
+                rc = BookieProtocol.EFENCED;
+            } catch (BookieException e) {
+                LOG.error("Unauthorized access to ledger {}", request.getLedgerId(), e);
+                rc = BookieProtocol.EUA;
+            } catch (Throwable t) {
+                LOG.error("Unexpected exception while writing {}@{} : {} ",
+                    request.getLedgerId(), request.getEntryId(), t.getMessage(), t);
+                rc = BookieProtocol.EBADREQ;
+            }
+
+            if (rc != BookieProtocol.EOK) {
+                hasFailedRequests = true;
+                requestStats.getAddEntryStats()
+                    .registerFailedEvent(MathUtils.elapsedNanos(requestNans), TimeUnit.NANOSECONDS);
+                cb.writeComplete(rc, request.getLedgerId(), request.getEntryId(), null, ctx);
+                iter.remove();
+                request.release();
+                request.recycle();
+            }
+        }
+        handleMap.clear();
+
+        if (hasFailedRequests && requestProcessor != null) {
+            requestProcessor.flushPendingResponses();
+        }
+
+        if (writeDataToJournal && !requests.isEmpty()) {

Review Comment:
   Whatever `writeDataToJournal` is true or false, we should release the request byteBuf.
   



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteBatchEntryProcessor.java:
##########
@@ -0,0 +1,143 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.proto;
+
+import static org.apache.bookkeeper.proto.BookieProtocol.ADDENTRY;
+
+import io.netty.util.Recycler;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieProtocol.ParsedAddRequest;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.util.MathUtils;
+
+/**
+ * Processes batched add entry requests.
+ */
+@Slf4j
+public class WriteBatchEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implements WriteCallback {
+    long startTimeNanos;
+    List<ParsedAddRequest> requests;
+    AtomicInteger requestCount;
+
+    @Override
+    protected void reset() {
+        requests = null;
+        requestHandler = null;
+        requestProcessor = null;
+        startTimeNanos = -1L;
+    }
+
+    public static WriteBatchEntryProcessor create(List<ParsedAddRequest> requests, BookieRequestHandler requestHandler,
+                                                  BookieRequestProcessor requestProcessor) {
+        WriteBatchEntryProcessor wbep = RECYCLER.get();
+        wbep.init(requests, requestHandler, requestProcessor);
+        requestProcessor.onAddRequestStart(requestHandler.ctx().channel(), requests.size());

Review Comment:
   In onAddRequestStart, it trackAddRequest once. The logic is different from before. 
   If there are 100 ParsedAddRequest:
   before: addsInProgress increase 100
   now: addsInProgress increase 1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3846: Streamline batch add request

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1137996730


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java:
##########
@@ -34,20 +40,24 @@
  */
 @Slf4j
 public class BookieRequestHandler extends ChannelInboundHandlerAdapter {
-
+    private static final int DEFAULT_CAPACITY = 1_000;
     static final Object EVENT_FLUSH_ALL_PENDING_RESPONSES = new Object();
 
     private final RequestProcessor requestProcessor;
     private final ChannelGroup allChannels;
 
     private ChannelHandlerContext ctx;
+    private final BlockingQueue<BookieProtocol.ParsedAddRequest> msgs;
 
     private ByteBuf pendingSendResponses = null;
     private int maxPendingResponsesSize;
 
     BookieRequestHandler(ServerConfiguration conf, RequestProcessor processor, ChannelGroup allChannels) {
         this.requestProcessor = processor;
         this.allChannels = allChannels;
+
+        int maxCapacity = conf.getMaxAddsInProgressLimit() > 0 ? conf.getMaxAddsInProgressLimit() : DEFAULT_CAPACITY;
+        this.msgs = new ArrayBlockingQueue<>(maxCapacity);

Review Comment:
   I think maybe not. The megs queue is also controlled by the Netty request handle. Netty also has a read batch control and once the batch limit is reached, it will call `channelReadComplete`, and we will flush the msgs queue requests to the storage when the `channelReadComplete` is called.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] dlg99 commented on a diff in pull request #3846: Streamline batch add request

Posted by "dlg99 (via GitHub)" <gi...@apache.org>.
dlg99 commented on code in PR #3846:
URL: https://github.com/apache/bookkeeper/pull/3846#discussion_r1142769155


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java:
##########
@@ -885,6 +887,26 @@ public void logAddEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb,
         logAddEntry(ledgerId, entryId, entry, ackBeforeSync, cb, ctx);
     }
 
+    public void logAddEntries(List<ByteBuf> entries, boolean ackBeforeSync, WriteCallback cb, Object ctx)
+        throws InterruptedException {
+        long reserveMemory = 0;
+        QueueEntry[] queueEntries = new QueueEntry[entries.size()];

Review Comment:
   can we pool these arrays?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java:
##########
@@ -205,21 +206,21 @@ public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, Stat
         readsSemaphore = maxReads > 0 ? new Semaphore(maxReads, true) : null;
     }
 
-    protected void onAddRequestStart(Channel channel) {
+    protected void onAddRequestStart(Channel channel, int permits) {
         if (addsSemaphore != null) {
-            if (!addsSemaphore.tryAcquire()) {
+            if (!addsSemaphore.tryAcquire(permits)) {

Review Comment:
   can try something like 
   ```
   int minPermits = 1;
   int available = addsSemaphore.availablePermits();
   permits = Math.min(Math.max(minPermits, available), permits)
   ```
   to reduce wait for permits. make onAddRequestStart return actual number of permits acquired, go from there.
   
   The risk is to drop to 1 almost all the time under load, but otherwise the load will be very spiky (switching from nothing to batch, creating more spiky load on the disk).
   This can be helped with permits > minPermits > 1
   
   have you tested with backpressure/maxAddsInProgressLimit enabled?
   



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java:
##########
@@ -205,21 +206,21 @@ public BookieRequestProcessor(ServerConfiguration serverCfg, Bookie bookie, Stat
         readsSemaphore = maxReads > 0 ? new Semaphore(maxReads, true) : null;
     }
 
-    protected void onAddRequestStart(Channel channel) {
+    protected void onAddRequestStart(Channel channel, int permits) {
         if (addsSemaphore != null) {
-            if (!addsSemaphore.tryAcquire()) {
+            if (!addsSemaphore.tryAcquire(permits)) {
                 final long throttlingStartTimeNanos = MathUtils.nowInNano();
                 channel.config().setAutoRead(false);
                 LOG.info("Too many add requests in progress, disabling autoread on channel {}", channel);
                 requestStats.blockAddRequest();
-                addsSemaphore.acquireUninterruptibly();
+                addsSemaphore.acquireUninterruptibly(permits);

Review Comment:
   permits must be <= maxAdds or this will never succeed.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org