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

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

Reply via email to