-
Notifications
You must be signed in to change notification settings - Fork 903
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
[Improve][Client] Group writing into the channel in PerChannelBookieClient #3886
base: master
Are you sure you want to change the base?
Changes from all commits
4cc7dbb
c1da2a3
b496c15
d5ad526
2417b4e
98b844d
ab95ba6
4ae8e53
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 |
---|---|---|
|
@@ -20,6 +20,9 @@ | |
|
||
import static org.apache.bookkeeper.client.LedgerHandle.INVALID_ENTRY_ID; | ||
|
||
import com.carrotsearch.hppc.ObjectHashSet; | ||
import com.carrotsearch.hppc.ObjectSet; | ||
import com.carrotsearch.hppc.procedures.ObjectProcedure; | ||
import com.google.common.base.Joiner; | ||
import com.google.common.collect.Sets; | ||
import com.google.protobuf.ByteString; | ||
|
@@ -173,6 +176,7 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter { | |
BKException.Code.DuplicateEntryIdException, | ||
BKException.Code.WriteOnReadOnlyBookieException)); | ||
private static final int DEFAULT_HIGH_PRIORITY_VALUE = 100; // We may add finer grained priority later. | ||
|
||
private static final AtomicLong txnIdGenerator = new AtomicLong(0); | ||
|
||
final BookieId bookieId; | ||
|
@@ -349,6 +353,9 @@ enum ConnectionState { | |
private final SecurityHandlerFactory shFactory; | ||
private volatile boolean isWritable = true; | ||
private long lastBookieUnavailableLogTimestamp = 0; | ||
private ByteBufList pendingSendRequests = null; | ||
private final ObjectSet<CompletionKey> pendingSendKeys = new ObjectHashSet<>(); | ||
private volatile boolean needFlush = true; | ||
|
||
public PerChannelBookieClient(OrderedExecutor executor, EventLoopGroup eventLoopGroup, | ||
BookieId addr, BookieAddressResolver bookieAddressResolver) throws SecurityException { | ||
|
@@ -1154,26 +1161,96 @@ private void writeAndFlush(final Channel channel, | |
} | ||
|
||
try { | ||
final long startTime = MathUtils.nowInNano(); | ||
|
||
ChannelPromise promise = channel.newPromise().addListener(future -> { | ||
if (future.isSuccess()) { | ||
nettyOpLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); | ||
CompletionValue completion = completionObjects.get(key); | ||
if (completion != null) { | ||
completion.setOutstanding(); | ||
} | ||
} else { | ||
nettyOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); | ||
if (request instanceof ByteBuf || request instanceof ByteBufList) { | ||
if (checkFlushPendingRequests(request)) { | ||
flushPendingRequests(); | ||
} | ||
}); | ||
channel.writeAndFlush(request, promise); | ||
prepareSendRequests(request, key); | ||
if (needFlush) { | ||
flushPendingRequests(); | ||
} | ||
} else { | ||
final long startTime = MathUtils.nowInNano(); | ||
|
||
// promise complete trigger flush pending request. | ||
ChannelPromise promise = channel.newPromise().addListener(future -> { | ||
if (future.isSuccess()) { | ||
nettyOpLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); | ||
CompletionValue completion = completionObjects.get(key); | ||
if (completion != null) { | ||
completion.setOutstanding(); | ||
} | ||
} else { | ||
nettyOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); | ||
} | ||
}); | ||
channel.writeAndFlush(request, promise); | ||
} | ||
} catch (Throwable e) { | ||
LOG.warn("Operation {} failed", StringUtils.requestToString(request), e); | ||
errorOut(key); | ||
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. The catch code block is not suitable for the new logic. It just handle the single key case, and didn't handle the pending keys case. |
||
} | ||
} | ||
|
||
public synchronized void prepareSendRequests(Object request, CompletionKey key) { | ||
if (pendingSendRequests == null) { | ||
if (request instanceof ByteBuf) { | ||
pendingSendRequests = ByteBufList.get((ByteBuf) request); | ||
} else if (request instanceof ByteBufList) { | ||
pendingSendRequests = ByteBufList.get((ByteBufList) request); | ||
} | ||
} else { | ||
BookieProtoEncoding.RequestEnDeCoderPreV3.serializeAddRequests(request, pendingSendRequests); | ||
} | ||
pendingSendKeys.add(key); | ||
} | ||
|
||
public synchronized boolean checkFlushPendingRequests(Object request) { | ||
if (pendingSendRequests == null) { | ||
return false; | ||
} | ||
|
||
int numBytes = request instanceof ByteBuf | ||
? ((ByteBuf) request).readableBytes() : ((ByteBufList) request).readableBytes(); | ||
return pendingSendRequests.readableBytes() + numBytes > maxFrameSize; | ||
} | ||
|
||
public synchronized void flushPendingRequests() { | ||
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. The flushPendingRequests trigger at every millisecond. It may exacerbate the lock race between 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. Here we can refer to the implements of |
||
if (pendingSendRequests == null) { | ||
needFlush = true; | ||
return; | ||
} | ||
|
||
final long startTime = MathUtils.nowInNano(); | ||
ObjectSet<CompletionKey> keys = new ObjectHashSet<>(pendingSendKeys); | ||
ChannelPromise promise = channel.newPromise().addListener(future -> { | ||
if (future.isSuccess()) { | ||
nettyOpLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); | ||
keys.forEach((ObjectProcedure<? super CompletionKey>) k -> { | ||
CompletionValue completion = completionObjects.get(k); | ||
if (completion != null) { | ||
completion.setOutstanding(); | ||
} | ||
}); | ||
|
||
} else { | ||
nettyOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); | ||
} | ||
flushPendingRequests(); | ||
}); | ||
|
||
if (channel != null && channel.isActive()) { | ||
channel.writeAndFlush(pendingSendRequests, promise); | ||
} else { | ||
pendingSendRequests.release(); | ||
pendingSendKeys.forEach((ObjectProcedure<? super CompletionKey>) key -> | ||
errorOut(key, BKException.Code.TooManyRequestsException)); | ||
} | ||
pendingSendRequests = null; | ||
pendingSendKeys.clear(); | ||
needFlush = false; | ||
} | ||
|
||
void errorOut(final CompletionKey key) { | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("Removing completion key: {}", key); | ||
|
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.
else throw IllegalStateException ?