-
Notifications
You must be signed in to change notification settings - Fork 360
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
[CELEBORN-1376] Push data failed should always release request body #2449
Changes from 12 commits
51c7cc7
e7fdfba
7d85518
ab29738
4846963
f8484c8
e2702fd
2bc8360
371af05
7750a6b
f448c4e
4d8f416
6e32149
c2613fc
f687735
f1986ec
b81e8f1
c5f067c
942946f
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 |
---|---|---|
|
@@ -208,14 +208,21 @@ public void sendRpc(ByteBuffer message) { | |
|
||
public ChannelFuture pushData( | ||
PushData pushData, long pushDataTimeout, RpcResponseCallback callback) { | ||
return pushData(pushData, pushDataTimeout, callback, null); | ||
Runnable rpcFailureCallback = | ||
new Runnable() { | ||
@Override | ||
public void run() { | ||
pushData.body().release(); | ||
} | ||
}; | ||
return pushData(pushData, pushDataTimeout, callback, rpcFailureCallback); | ||
} | ||
|
||
public ChannelFuture pushData( | ||
PushData pushData, | ||
long pushDataTimeout, | ||
RpcResponseCallback callback, | ||
Runnable rpcSendoutCallback) { | ||
Runnable rpcFailureCallback) { | ||
if (logger.isTraceEnabled()) { | ||
logger.trace("Pushing data to {}", NettyUtils.getRemoteAddress(channel)); | ||
} | ||
|
@@ -225,14 +232,29 @@ public ChannelFuture pushData( | |
PushRequestInfo info = new PushRequestInfo(dueTime, callback); | ||
handler.addPushRequest(requestId, info); | ||
pushData.requestId = requestId; | ||
PushChannelListener listener = new PushChannelListener(requestId, rpcSendoutCallback); | ||
PushChannelListener listener = new PushChannelListener(requestId, rpcFailureCallback); | ||
ChannelFuture channelFuture = channel.writeAndFlush(pushData).addListener(listener); | ||
info.setChannelFuture(channelFuture); | ||
return channelFuture; | ||
} | ||
|
||
public ChannelFuture pushMergedData( | ||
PushMergedData pushMergedData, long pushDataTimeout, RpcResponseCallback callback) { | ||
Runnable rpcFailureCallback = | ||
new Runnable() { | ||
@Override | ||
public void run() { | ||
pushMergedData.body().release(); | ||
} | ||
}; | ||
return pushMergedData(pushMergedData, pushDataTimeout, callback, rpcFailureCallback); | ||
} | ||
|
||
public ChannelFuture pushMergedData( | ||
PushMergedData pushMergedData, | ||
long pushDataTimeout, | ||
RpcResponseCallback callback, | ||
Runnable rpcFailureCallback) { | ||
if (logger.isTraceEnabled()) { | ||
logger.trace("Pushing merged data to {}", NettyUtils.getRemoteAddress(channel)); | ||
} | ||
|
@@ -243,7 +265,7 @@ public ChannelFuture pushMergedData( | |
handler.addPushRequest(requestId, info); | ||
pushMergedData.requestId = requestId; | ||
|
||
PushChannelListener listener = new PushChannelListener(requestId); | ||
PushChannelListener listener = new PushChannelListener(requestId, rpcFailureCallback); | ||
ChannelFuture channelFuture = channel.writeAndFlush(pushMergedData).addListener(listener); | ||
info.setChannelFuture(channelFuture); | ||
return channelFuture; | ||
|
@@ -415,29 +437,29 @@ protected void handleFailure(String errorMsg, Throwable cause) { | |
|
||
private class PushChannelListener extends StdChannelListener { | ||
final long pushRequestId; | ||
Runnable rpcSendOutCallback; | ||
Runnable rpcFailureCallback; | ||
|
||
PushChannelListener(long pushRequestId) { | ||
this(pushRequestId, null); | ||
} | ||
|
||
PushChannelListener(long pushRequestId, Runnable rpcSendOutCallback) { | ||
PushChannelListener(long pushRequestId, Runnable rpcFailureCallback) { | ||
super("PUSH " + pushRequestId); | ||
this.pushRequestId = pushRequestId; | ||
this.rpcSendOutCallback = rpcSendOutCallback; | ||
this.rpcFailureCallback = rpcFailureCallback; | ||
} | ||
|
||
@Override | ||
public void operationComplete(Future<? super Void> future) throws Exception { | ||
super.operationComplete(future); | ||
if (rpcSendOutCallback != null) { | ||
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. 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. IIRC this will affect buffer release from flink, IMO we can retain the current setup and remove the rpcFailureCallback call from handleFailure, since operationComplete will be invoked regardless of whether the channel fails 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. How about current? 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.
Make sense. |
||
rpcSendOutCallback.run(); | ||
} | ||
} | ||
|
||
@Override | ||
protected void handleFailure(String errorMsg, Throwable cause) { | ||
handler.handlePushFailure(pushRequestId, errorMsg, cause); | ||
if (rpcFailureCallback != null) { | ||
rpcFailureCallback.run(); | ||
} | ||
} | ||
} | ||
} |
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 an outbound (a.k.a. downstream) message reaches at the beginning of the pipeline, Netty will release it after writing it out." So why is pushdata not released when rpc failure occurs?
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.
@FMX
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.
That's because in this case error/Exception occurs before data is sent to pipeline.
future.isSuccess()
returns false inStdChannelListener#operationComplete