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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to