You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/06/22 16:05:17 UTC

[GitHub] [pulsar] merlimat commented on a diff in pull request #15955: PIP-174: New managed ledger entry cache implementation

merlimat commented on code in PR #15955:
URL: https://github.com/apache/pulsar/pull/15955#discussion_r903958286


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/SharedCacheSegment.java:
##########
@@ -0,0 +1,94 @@
+/**
+ * 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.mledger.impl.cache;
+
+import io.netty.buffer.ByteBuf;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap;
+
+class SharedCacheSegment implements AutoCloseable {
+
+    private final ByteBuf cacheBuffer;
+    private final AtomicInteger currentOffset = new AtomicInteger();
+    private final ConcurrentLongLongPairHashMap index;
+    private final int segmentSize;
+
+    private static final int ALIGN_64_MASK = ~(64 - 1);
+
+    SharedCacheSegment(int segmentSize) {
+        this.segmentSize = segmentSize;
+        this.cacheBuffer = PulsarByteBufAllocator.DEFAULT.buffer(segmentSize, segmentSize);
+        this.index = ConcurrentLongLongPairHashMap.newBuilder()
+                // We are going to often clear() the map, with the expectation that it's going to get filled again
+                // immediately after. In these conditions it does not make sense to shrink it each time.
+                .autoShrink(false)
+                .concurrencyLevel(Runtime.getRuntime().availableProcessors() * 2)
+                .build();
+    }
+
+    boolean insert(long ledgerId, long entryId, ByteBuf entry) {
+        int entrySize = entry.readableBytes();
+        int alignedSize = align64(entrySize);
+        int offset = currentOffset.getAndAdd(alignedSize);
+
+        if (offset + entrySize > segmentSize) {
+            // The segment is full
+            return false;
+        } else {
+            // Copy entry into read cache segment
+            cacheBuffer.setBytes(offset, entry, entry.readerIndex(), entry.readableBytes());
+            long value = offset << 32 | entrySize;
+            index.put(ledgerId, entryId, value, 0);
+            return true;
+        }
+    }
+
+    public ByteBuf get(long ledgerId, long entryId) {
+        long value = index.getFirstValue(ledgerId, entryId);
+        if (value >= 0) {
+            int offset = (int) (value >> 32);
+            int entryLen = (int) value;
+
+            ByteBuf entry = PulsarByteBufAllocator.DEFAULT.buffer(entryLen, entryLen);

Review Comment:
   Yes, we could return a "retained slice" (a `ByteBuf` that increments the ref-count and points to a portion of the original buffer) and avoid the copy on the read path. 
   The problem would be that this buffer could stay alive for an indefinite amount of time, in the case of some consumer connections being slow. With this, we'd be retaining a whole 1GB buffer even if a small message is pending on a TCP connection, and we cannot just overwrite the old cache segment when rotating because the reader could still be there.
   
   Since we already have a flag to control the copy/not-copy of the cache, another approach I was thinking of was to keep maps of the original ByteBuf (so that we also eliminate the copy on insertion in the cache). 
   We still do the rotation based on rotating the segments, where each segment has its own hash map.



-- 
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@pulsar.apache.org

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