Skip to content

Commit 234b817

Browse files
merlimathangc0276
andauthored
Single buffer for small add requests (#3783)
* Single buffer for small add requests * Fixed checkstyle * Fixed treating of ComposityByteBuf * Fixed merge issues * Fixed merge issues * WIP * Fixed test and removed dead code * Removed unused import * Fixed BookieJournalTest * removed unused import * fix the checkstyle * fix failed test * fix failed test --------- Co-authored-by: chenhang <[email protected]>
1 parent e19cb9d commit 234b817

20 files changed

+187
-151
lines changed

bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java

Lines changed: 13 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,9 @@
2929
import static org.apache.bookkeeper.replication.ReplicationStats.WRITE_DATA_LATENCY;
3030

3131
import com.google.common.util.concurrent.RateLimiter;
32+
import io.netty.buffer.ByteBuf;
3233
import io.netty.buffer.Unpooled;
34+
import io.netty.util.ReferenceCounted;
3335
import java.util.Enumeration;
3436
import java.util.HashSet;
3537
import java.util.Iterator;
@@ -403,17 +405,24 @@ public void readComplete(int rc, LedgerHandle lh,
403405
numEntriesRead.inc();
404406
numBytesRead.registerSuccessfulValue(dataLength);
405407

406-
ByteBufList toSend = lh.getDigestManager()
408+
ReferenceCounted toSend = lh.getDigestManager()
407409
.computeDigestAndPackageForSending(entryId,
408410
lh.getLastAddConfirmed(), entry.getLength(),
409-
Unpooled.wrappedBuffer(data, 0, data.length));
411+
Unpooled.wrappedBuffer(data, 0, data.length),
412+
lh.getLedgerKey(),
413+
0
414+
);
410415
if (replicationThrottle != null) {
411-
updateAverageEntrySize(toSend.readableBytes());
416+
if (toSend instanceof ByteBuf) {
417+
updateAverageEntrySize(((ByteBuf) toSend).readableBytes());
418+
} else if (toSend instanceof ByteBufList) {
419+
updateAverageEntrySize(((ByteBufList) toSend).readableBytes());
420+
}
412421
}
413422
for (BookieId newBookie : newBookies) {
414423
long startWriteEntryTime = MathUtils.nowInNano();
415424
bkc.getBookieClient().addEntry(newBookie, lh.getId(),
416-
lh.getLedgerKey(), entryId, ByteBufList.clone(toSend),
425+
lh.getLedgerKey(), entryId, toSend,
417426
multiWriteCallback, dataLength, BookieProtocol.FLAG_RECOVERY_ADD,
418427
false, WriteFlag.NONE);
419428
writeDataLatency.registerSuccessfulEvent(

bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import io.netty.util.Recycler;
2929
import io.netty.util.Recycler.Handle;
3030
import io.netty.util.ReferenceCountUtil;
31+
import io.netty.util.ReferenceCounted;
3132
import java.util.EnumSet;
3233
import java.util.HashSet;
3334
import java.util.List;
@@ -38,7 +39,6 @@
3839
import org.apache.bookkeeper.client.api.WriteFlag;
3940
import org.apache.bookkeeper.net.BookieId;
4041
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
41-
import org.apache.bookkeeper.util.ByteBufList;
4242
import org.apache.bookkeeper.util.MathUtils;
4343
import org.slf4j.Logger;
4444
import org.slf4j.LoggerFactory;
@@ -56,7 +56,7 @@ class PendingAddOp implements WriteCallback {
5656
private static final Logger LOG = LoggerFactory.getLogger(PendingAddOp.class);
5757

5858
ByteBuf payload;
59-
ByteBufList toSend;
59+
ReferenceCounted toSend;
6060
AddCallbackWithLatency cb;
6161
Object ctx;
6262
long entryId;
@@ -242,9 +242,10 @@ public synchronized void initiate() {
242242
checkNotNull(lh);
243243
checkNotNull(lh.macManager);
244244

245+
int flags = isRecoveryAdd ? FLAG_RECOVERY_ADD | FLAG_HIGH_PRIORITY : FLAG_NONE;
245246
this.toSend = lh.macManager.computeDigestAndPackageForSending(
246247
entryId, lh.lastAddConfirmed, currentLedgerLength,
247-
payload);
248+
payload, lh.ledgerKey, flags);
248249
// ownership of RefCounted ByteBuf was passed to computeDigestAndPackageForSending
249250
payload = null;
250251

bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import static org.apache.bookkeeper.auth.AuthProviderFactoryFactory.AUTHENTICATION_DISABLED_PLUGIN_NAME;
2424

2525
import com.google.protobuf.ByteString;
26+
import io.netty.buffer.ByteBuf;
2627
import io.netty.channel.Channel;
2728
import io.netty.channel.ChannelDuplexHandler;
2829
import io.netty.channel.ChannelHandlerContext;
@@ -39,6 +40,7 @@
3940
import org.apache.bookkeeper.auth.ClientAuthProvider;
4041
import org.apache.bookkeeper.client.BKException;
4142
import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage;
43+
import org.apache.bookkeeper.util.ByteBufList;
4244
import org.slf4j.Logger;
4345
import org.slf4j.LoggerFactory;
4446

@@ -358,8 +360,10 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
358360
} else {
359361
waitingForAuth.add(msg);
360362
}
363+
} else if (msg instanceof ByteBuf || msg instanceof ByteBufList) {
364+
waitingForAuth.add(msg);
361365
} else {
362-
LOG.info("dropping write of message {}", msg);
366+
LOG.info("[{}] dropping write of message {}", ctx.channel(), msg);
363367
}
364368
}
365369
}

bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
*/
2121
package org.apache.bookkeeper.proto;
2222

23+
import io.netty.util.ReferenceCounted;
2324
import java.util.EnumSet;
2425
import java.util.List;
2526
import java.util.concurrent.CompletableFuture;
@@ -139,7 +140,7 @@ void writeLac(BookieId address, long ledgerId, byte[] masterKey,
139140
* {@link org.apache.bookkeeper.client.api.WriteFlag}
140141
*/
141142
void addEntry(BookieId address, long ledgerId, byte[] masterKey,
142-
long entryId, ByteBufList toSend, WriteCallback cb, Object ctx,
143+
long entryId, ReferenceCounted toSend, WriteCallback cb, Object ctx,
143144
int options, boolean allowFastFail, EnumSet<WriteFlag> writeFlags);
144145

145146
/**

bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import io.netty.util.Recycler;
3333
import io.netty.util.Recycler.Handle;
3434
import io.netty.util.ReferenceCountUtil;
35+
import io.netty.util.ReferenceCounted;
3536
import io.netty.util.concurrent.DefaultThreadFactory;
3637
import java.io.IOException;
3738
import java.util.EnumSet;
@@ -288,7 +289,7 @@ public void addEntry(final BookieId addr,
288289
final long ledgerId,
289290
final byte[] masterKey,
290291
final long entryId,
291-
final ByteBufList toSend,
292+
final ReferenceCounted toSend,
292293
final WriteCallback cb,
293294
final Object ctx,
294295
final int options,
@@ -357,7 +358,7 @@ private static class ChannelReadyForAddEntryCallback
357358
private final Handle<ChannelReadyForAddEntryCallback> recyclerHandle;
358359

359360
private BookieClientImpl bookieClient;
360-
private ByteBufList toSend;
361+
private ReferenceCounted toSend;
361362
private long ledgerId;
362363
private long entryId;
363364
private BookieId addr;
@@ -369,7 +370,7 @@ private static class ChannelReadyForAddEntryCallback
369370
private EnumSet<WriteFlag> writeFlags;
370371

371372
static ChannelReadyForAddEntryCallback create(
372-
BookieClientImpl bookieClient, ByteBufList toSend, long ledgerId,
373+
BookieClientImpl bookieClient, ReferenceCounted toSend, long ledgerId,
373374
long entryId, BookieId addr, Object ctx,
374375
WriteCallback cb, int options, byte[] masterKey, boolean allowFastFail,
375376
EnumSet<WriteFlag> writeFlags) {

bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java

Lines changed: 4 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -110,23 +110,7 @@ public Object encode(Object msg, ByteBufAllocator allocator)
110110
return msg;
111111
}
112112
BookieProtocol.Request r = (BookieProtocol.Request) msg;
113-
if (r instanceof BookieProtocol.AddRequest) {
114-
BookieProtocol.AddRequest ar = (BookieProtocol.AddRequest) r;
115-
ByteBufList data = ar.getData();
116-
117-
int totalHeaderSize = 4 // for the request header
118-
+ BookieProtocol.MASTER_KEY_LENGTH; // for the master key
119-
120-
int totalPayloadSize = totalHeaderSize + data.readableBytes();
121-
ByteBuf buf = allocator.buffer(totalHeaderSize + 4 /* frame size */);
122-
buf.writeInt(totalPayloadSize); // Frame header
123-
buf.writeInt(PacketHeader.toInt(r.getProtocolVersion(), r.getOpCode(), r.getFlags()));
124-
buf.writeBytes(r.getMasterKey(), 0, BookieProtocol.MASTER_KEY_LENGTH);
125-
126-
ar.recycle();
127-
data.prepend(buf);
128-
return data;
129-
} else if (r instanceof BookieProtocol.ReadRequest) {
113+
if (r instanceof BookieProtocol.ReadRequest) {
130114
int totalHeaderSize = 4 // for request type
131115
+ 8 // for ledgerId
132116
+ 8; // for entryId
@@ -437,7 +421,9 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
437421
if (LOG.isTraceEnabled()) {
438422
LOG.trace("Encode request {} to channel {}.", msg, ctx.channel());
439423
}
440-
if (msg instanceof BookkeeperProtocol.Request) {
424+
if (msg instanceof ByteBuf || msg instanceof ByteBufList) {
425+
ctx.write(msg, promise);
426+
} else if (msg instanceof BookkeeperProtocol.Request) {
441427
ctx.write(reqV3.encode(msg, ctx.alloc()), promise);
442428
} else if (msg instanceof BookieProtocol.Request) {
443429
ctx.write(reqPreV3.encode(msg, ctx.alloc()), promise);

bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java

Lines changed: 0 additions & 53 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@
2727
import io.netty.util.ReferenceCountUtil;
2828
import io.netty.util.ReferenceCounted;
2929
import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage;
30-
import org.apache.bookkeeper.util.ByteBufList;
3130

3231
/**
3332
* The packets of the Bookie protocol all have a 4-byte integer indicating the
@@ -252,58 +251,6 @@ public String toString() {
252251
public void recycle() {}
253252
}
254253

255-
/**
256-
* A Request that adds data.
257-
*/
258-
class AddRequest extends Request {
259-
ByteBufList data;
260-
261-
static AddRequest create(byte protocolVersion, long ledgerId,
262-
long entryId, short flags, byte[] masterKey,
263-
ByteBufList data) {
264-
AddRequest add = RECYCLER.get();
265-
add.protocolVersion = protocolVersion;
266-
add.opCode = ADDENTRY;
267-
add.ledgerId = ledgerId;
268-
add.entryId = entryId;
269-
add.flags = flags;
270-
add.masterKey = masterKey;
271-
add.data = data.retain();
272-
return add;
273-
}
274-
275-
ByteBufList getData() {
276-
// We need to have different ByteBufList instances for each bookie write
277-
return ByteBufList.clone(data);
278-
}
279-
280-
boolean isRecoveryAdd() {
281-
return (flags & FLAG_RECOVERY_ADD) == FLAG_RECOVERY_ADD;
282-
}
283-
284-
private final Handle<AddRequest> recyclerHandle;
285-
private AddRequest(Handle<AddRequest> recyclerHandle) {
286-
this.recyclerHandle = recyclerHandle;
287-
}
288-
289-
private static final Recycler<AddRequest> RECYCLER = new Recycler<AddRequest>() {
290-
@Override
291-
protected AddRequest newObject(Handle<AddRequest> handle) {
292-
return new AddRequest(handle);
293-
}
294-
};
295-
296-
@Override
297-
public void recycle() {
298-
ledgerId = -1;
299-
entryId = -1;
300-
masterKey = null;
301-
ReferenceCountUtil.release(data);
302-
data = null;
303-
recyclerHandle.recycle(this);
304-
}
305-
}
306-
307254
/**
308255
* This is similar to add request, but it used when processing the request on the bookie side.
309256
*/

bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java

Lines changed: 15 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -62,6 +62,7 @@
6262
import io.netty.util.Recycler;
6363
import io.netty.util.Recycler.Handle;
6464
import io.netty.util.ReferenceCountUtil;
65+
import io.netty.util.ReferenceCounted;
6566
import io.netty.util.concurrent.Future;
6667
import io.netty.util.concurrent.GenericFutureListener;
6768
import java.io.IOException;
@@ -771,7 +772,7 @@ void forceLedger(final long ledgerId, ForceLedgerCallback cb, Object ctx) {
771772
* @param writeFlags
772773
* WriteFlags
773774
*/
774-
void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBufList toSend, WriteCallback cb,
775+
void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ReferenceCounted toSend, WriteCallback cb,
775776
Object ctx, final int options, boolean allowFastFail, final EnumSet<WriteFlag> writeFlags) {
776777
Object request = null;
777778
CompletionKey completionKey = null;
@@ -782,9 +783,12 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf
782783
return;
783784
}
784785
completionKey = acquireV2Key(ledgerId, entryId, OperationType.ADD_ENTRY);
785-
request = BookieProtocol.AddRequest.create(
786-
BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, entryId,
787-
(short) options, masterKey, toSend);
786+
787+
if (toSend instanceof ByteBuf) {
788+
request = ((ByteBuf) toSend).retainedDuplicate();
789+
} else {
790+
request = ByteBufList.clone((ByteBufList) toSend);
791+
}
788792
} else {
789793
final long txnId = getTxnId();
790794
completionKey = new V3CompletionKey(txnId, OperationType.ADD_ENTRY);
@@ -799,11 +803,14 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf
799803
}
800804

801805
ByteString body = null;
802-
if (toSend.hasArray()) {
803-
body = UnsafeByteOperations.unsafeWrap(toSend.array(), toSend.arrayOffset(), toSend.readableBytes());
806+
ByteBufList bufToSend = (ByteBufList) toSend;
807+
808+
if (bufToSend.hasArray()) {
809+
body = UnsafeByteOperations.unsafeWrap(bufToSend.array(), bufToSend.arrayOffset(),
810+
bufToSend.readableBytes());
804811
} else {
805-
for (int i = 0; i < toSend.size(); i++) {
806-
ByteString piece = UnsafeByteOperations.unsafeWrap(toSend.getBuffer(i).nioBuffer());
812+
for (int i = 0; i < bufToSend.size(); i++) {
813+
ByteString piece = UnsafeByteOperations.unsafeWrap(bufToSend.getBuffer(i).nioBuffer());
807814
// use ByteString.concat to avoid byte[] allocation when toSend has multiple ByteBufs
808815
body = (body == null) ? piece : body.concat(piece);
809816
}
@@ -1143,14 +1150,6 @@ private void writeAndFlush(final Channel channel,
11431150
StringUtils.requestToString(request));
11441151

11451152
errorOut(key, BKException.Code.TooManyRequestsException);
1146-
1147-
// If the request is a V2 add request, we retained the data's reference when creating the AddRequest
1148-
// object. To avoid the object leak, we need to release the reference if we met any errors
1149-
// before sending it.
1150-
if (request instanceof BookieProtocol.AddRequest) {
1151-
BookieProtocol.AddRequest ar = (BookieProtocol.AddRequest) request;
1152-
ar.recycle();
1153-
}
11541153
return;
11551154
}
11561155

0 commit comments

Comments
 (0)