-
Notifications
You must be signed in to change notification settings - Fork 907
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Single buffer for small add requests #3783
Changes from 2 commits
11d5854
4fd46b0
7901773
572310c
2b6632f
5b50e06
62c0bdf
cc3c343
8791faf
ca4a555
e584323
28d3581
3390bc5
822eb3e
eeedef6
76c4768
441f522
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,10 +22,13 @@ | |
import io.netty.buffer.CompositeByteBuf; | ||
import io.netty.buffer.Unpooled; | ||
import io.netty.util.ReferenceCountUtil; | ||
import io.netty.util.ReferenceCounted; | ||
import java.security.GeneralSecurityException; | ||
import java.security.NoSuchAlgorithmException; | ||
import org.apache.bookkeeper.client.BKException.BKDigestMatchException; | ||
import org.apache.bookkeeper.client.LedgerHandle; | ||
import org.apache.bookkeeper.proto.BookieProtoEncoding; | ||
import org.apache.bookkeeper.proto.BookieProtocol; | ||
import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; | ||
import org.apache.bookkeeper.util.ByteBufList; | ||
import org.slf4j.Logger; | ||
|
@@ -96,14 +99,61 @@ public static byte[] generateMasterKey(byte[] password) throws NoSuchAlgorithmEx | |
* @param data | ||
* @return | ||
*/ | ||
public ByteBufList computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, long length, | ||
ByteBuf data) { | ||
ByteBuf headersBuffer; | ||
public ReferenceCounted computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, long length, | ||
ByteBuf data, byte[] masterKey, int flags) { | ||
if (this.useV2Protocol) { | ||
headersBuffer = allocator.buffer(METADATA_LENGTH + macCodeLength); | ||
return computeDigestAndPackageForSendingV2(entryId, lastAddConfirmed, length, data, masterKey, flags); | ||
} else { | ||
headersBuffer = Unpooled.buffer(METADATA_LENGTH + macCodeLength); | ||
return computeDigestAndPackageForSendingV3(entryId, lastAddConfirmed, length, data); | ||
} | ||
} | ||
|
||
private ReferenceCounted computeDigestAndPackageForSendingV2(long entryId, long lastAddConfirmed, long length, | ||
ByteBuf data, byte[] masterKey, int flags) { | ||
boolean isSmallEntry = data.readableBytes() < BookieProtoEncoding.SMALL_ENTRY_SIZE_THRESHOLD; | ||
|
||
int headersSize = 4 // Request header | ||
+ BookieProtocol.MASTER_KEY_LENGTH // for the master key | ||
+ METADATA_LENGTH // | ||
+ macCodeLength; | ||
int payloadSize = data.readableBytes(); | ||
int bufferSize = 4 + headersSize + (isSmallEntry ? payloadSize : 0); | ||
|
||
ByteBuf buf = allocator.buffer(bufferSize, bufferSize); | ||
buf.writeInt(headersSize + payloadSize); | ||
buf.writeInt( | ||
BookieProtocol.PacketHeader.toInt( | ||
BookieProtocol.CURRENT_PROTOCOL_VERSION, BookieProtocol.ADDENTRY, (short) flags)); | ||
buf.writeBytes(masterKey, 0, BookieProtocol.MASTER_KEY_LENGTH); | ||
|
||
// The checksum is computed on the next part of the buffer only | ||
buf.readerIndex(buf.writerIndex()); | ||
buf.writeLong(ledgerId); | ||
buf.writeLong(entryId); | ||
buf.writeLong(lastAddConfirmed); | ||
buf.writeLong(length); | ||
|
||
// Compute checksum over the headers | ||
update(buf); | ||
update(data); | ||
|
||
populateValueAndReset(buf); | ||
|
||
// Reset the reader index to the beginning | ||
buf.readerIndex(0); | ||
|
||
if (isSmallEntry) { | ||
buf.writeBytes(data); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. buf.writeBytes(unwrap) ? |
||
data.release(); | ||
return buf; | ||
} else { | ||
return ByteBufList.get(buf, data); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ByteBufList.get(buf, unwrap)? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 |
||
} | ||
} | ||
|
||
private ByteBufList computeDigestAndPackageForSendingV3(long entryId, long lastAddConfirmed, long length, | ||
ByteBuf data) { | ||
ByteBuf headersBuffer = Unpooled.buffer(METADATA_LENGTH + macCodeLength); | ||
headersBuffer.writeLong(ledgerId); | ||
headersBuffer.writeLong(entryId); | ||
headersBuffer.writeLong(lastAddConfirmed); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to unwrap the data ByteBuf if it is CompositeByteBuf?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think for v2 protocol, we shouldn't be getting any ComposityByteBuf
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When Pulsar enabled KOP AppendIndexMetadataInterceptor, the payload will be CompositeByteBuf, and the
update(data)
will copy data to the heap. Please refer to: #2701