-
Notifications
You must be signed in to change notification settings - Fork 368
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-1757] Add retry when sending RPC to LifecycleManager #3008
base: main
Are you sure you want to change the base?
Changes from 18 commits
a928c40
591ad34
b636fbb
b63c232
d968c77
4b98374
42687e7
0a4b8b7
0364bd3
0999748
dce8f13
99154e8
5bcf9fc
f8cd555
bc6237a
12650d1
b0d6e58
f515888
0451af2
015fbb3
f317d6d
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,10 +20,7 @@ | |
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
import java.util.*; | ||
import java.util.concurrent.Callable; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.*; | ||
|
||
import scala.Tuple2; | ||
import scala.reflect.ClassTag$; | ||
|
@@ -81,6 +78,7 @@ public class ShuffleClientImpl extends ShuffleClient { | |
|
||
private final int registerShuffleMaxRetries; | ||
private final long registerShuffleRetryWaitMs; | ||
private final int callLifecycleManagerMaxRetry; | ||
private final int maxReviveTimes; | ||
private final boolean testRetryRevive; | ||
private final int pushBufferMaxSize; | ||
|
@@ -179,6 +177,7 @@ public ShuffleClientImpl(String appUniqueId, CelebornConf conf, UserIdentifier u | |
this.userIdentifier = userIdentifier; | ||
registerShuffleMaxRetries = conf.clientRegisterShuffleMaxRetry(); | ||
registerShuffleRetryWaitMs = conf.clientRegisterShuffleRetryWaitMs(); | ||
callLifecycleManagerMaxRetry = conf.clientCallLifecycleManagerMaxRetry(); | ||
maxReviveTimes = conf.clientPushMaxReviveTimes(); | ||
testRetryRevive = conf.testRetryRevive(); | ||
pushBufferMaxSize = conf.clientPushBufferMaxSize(); | ||
|
@@ -534,6 +533,7 @@ private ConcurrentHashMap<Integer, PartitionLocation> registerShuffle( | |
lifecycleManagerRef.askSync( | ||
RegisterShuffle$.MODULE$.apply(shuffleId, numMappers, numPartitions), | ||
conf.clientRpcRegisterShuffleAskTimeout(), | ||
callLifecycleManagerMaxRetry, | ||
ClassTag$.MODULE$.apply(PbRegisterShuffleResponse.class))); | ||
} | ||
|
||
|
@@ -1700,13 +1700,12 @@ private void mapEndInternal( | |
throws IOException { | ||
final String mapKey = Utils.makeMapKey(shuffleId, mapId, attemptId); | ||
PushState pushState = getPushState(mapKey); | ||
|
||
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. unnecessary change. |
||
try { | ||
limitZeroInFlight(mapKey, pushState); | ||
|
||
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. unnecessary change. |
||
MapperEndResponse response = | ||
lifecycleManagerRef.askSync( | ||
new MapperEnd(shuffleId, mapId, attemptId, numMappers, partitionId), | ||
callLifecycleManagerMaxRetry, | ||
ClassTag$.MODULE$.apply(MapperEndResponse.class)); | ||
if (response.status() != StatusCode.SUCCESS) { | ||
throw new CelebornIOException("MapperEnd failed! StatusCode: " + response.status()); | ||
|
@@ -1741,65 +1740,60 @@ public boolean cleanupShuffle(int shuffleId) { | |
|
||
protected Tuple2<ReduceFileGroups, String> loadFileGroupInternal( | ||
int shuffleId, boolean isSegmentGranularityVisible) { | ||
{ | ||
long getReducerFileGroupStartTime = System.nanoTime(); | ||
String exceptionMsg = null; | ||
try { | ||
if (lifecycleManagerRef == null) { | ||
exceptionMsg = "Driver endpoint is null!"; | ||
logger.warn(exceptionMsg); | ||
} else { | ||
GetReducerFileGroup getReducerFileGroup = | ||
new GetReducerFileGroup(shuffleId, isSegmentGranularityVisible); | ||
|
||
GetReducerFileGroupResponse response = | ||
lifecycleManagerRef.askSync( | ||
getReducerFileGroup, | ||
conf.clientRpcGetReducerFileGroupAskTimeout(), | ||
ClassTag$.MODULE$.apply(GetReducerFileGroupResponse.class)); | ||
long getReducerFileGroupStartTime = System.nanoTime(); | ||
String exceptionMsg = null; | ||
if (lifecycleManagerRef == null) { | ||
exceptionMsg = "Driver endpoint is null!"; | ||
logger.warn(exceptionMsg); | ||
return Tuple2.apply(null, exceptionMsg); | ||
} | ||
try { | ||
GetReducerFileGroup getReducerFileGroup = | ||
new GetReducerFileGroup(shuffleId, isSegmentGranularityVisible); | ||
|
||
switch (response.status()) { | ||
case SUCCESS: | ||
logger.info( | ||
"Shuffle {} request reducer file group success using {} ms, result partition size {}.", | ||
shuffleId, | ||
TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - getReducerFileGroupStartTime), | ||
response.fileGroup().size()); | ||
return Tuple2.apply( | ||
new ReduceFileGroups( | ||
response.fileGroup(), response.attempts(), response.partitionIds()), | ||
null); | ||
case SHUFFLE_NOT_REGISTERED: | ||
logger.warn( | ||
"Request {} return {} for {}.", | ||
getReducerFileGroup, | ||
response.status(), | ||
shuffleId); | ||
// return empty result | ||
return Tuple2.apply( | ||
new ReduceFileGroups( | ||
response.fileGroup(), response.attempts(), response.partitionIds()), | ||
null); | ||
case STAGE_END_TIME_OUT: | ||
case SHUFFLE_DATA_LOST: | ||
exceptionMsg = | ||
String.format( | ||
"Request %s return %s for %s.", | ||
getReducerFileGroup, response.status(), shuffleId); | ||
logger.warn(exceptionMsg); | ||
break; | ||
default: // fall out | ||
} | ||
} | ||
} catch (Exception e) { | ||
if (e instanceof InterruptedException) { | ||
Thread.currentThread().interrupt(); | ||
} | ||
logger.error("Exception raised while call GetReducerFileGroup for {}.", shuffleId, e); | ||
exceptionMsg = e.getMessage(); | ||
GetReducerFileGroupResponse response = | ||
lifecycleManagerRef.askSync( | ||
getReducerFileGroup, | ||
conf.clientRpcGetReducerFileGroupAskTimeout(), | ||
callLifecycleManagerMaxRetry, | ||
ClassTag$.MODULE$.apply(GetReducerFileGroupResponse.class)); | ||
switch (response.status()) { | ||
case SUCCESS: | ||
logger.info( | ||
"Shuffle {} request reducer file group success using {} ms, result partition size {}.", | ||
shuffleId, | ||
TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - getReducerFileGroupStartTime), | ||
response.fileGroup().size()); | ||
return Tuple2.apply( | ||
new ReduceFileGroups( | ||
response.fileGroup(), response.attempts(), response.partitionIds()), | ||
null); | ||
case SHUFFLE_NOT_REGISTERED: | ||
logger.warn( | ||
"Request {} return {} for {}.", getReducerFileGroup, response.status(), shuffleId); | ||
// return empty result | ||
return Tuple2.apply( | ||
new ReduceFileGroups( | ||
response.fileGroup(), response.attempts(), response.partitionIds()), | ||
null); | ||
case STAGE_END_TIME_OUT: | ||
case SHUFFLE_DATA_LOST: | ||
exceptionMsg = | ||
String.format( | ||
"Request %s return %s for %s.", | ||
getReducerFileGroup, response.status(), shuffleId); | ||
logger.warn(exceptionMsg); | ||
break; | ||
default: // fall out | ||
} | ||
return Tuple2.apply(null, exceptionMsg); | ||
} catch (Exception e) { | ||
if (e instanceof InterruptedException) { | ||
Thread.currentThread().interrupt(); | ||
} | ||
logger.error("Exception raised while call GetReducerFileGroup for {}.", shuffleId, e); | ||
exceptionMsg = e.getMessage(); | ||
} | ||
return Tuple2.apply(null, exceptionMsg); | ||
} | ||
|
||
@Override | ||
|
@@ -1929,7 +1923,10 @@ public void shutdown() { | |
public void setupLifecycleManagerRef(String host, int port) { | ||
logger.info("setupLifecycleManagerRef: host = {}, port = {}", host, port); | ||
lifecycleManagerRef = | ||
rpcEnv.setupEndpointRef(new RpcAddress(host, port), RpcNameConstants.LIFECYCLE_MANAGER_EP); | ||
rpcEnv.setupEndpointRef( | ||
new RpcAddress(host, port), | ||
RpcNameConstants.LIFECYCLE_MANAGER_EP, | ||
callLifecycleManagerMaxRetry); | ||
initDataClientFactoryIfNeeded(); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -520,6 +520,7 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se | |
new RpcTimeout(get(RPC_LOOKUP_TIMEOUT).milli, RPC_LOOKUP_TIMEOUT.key) | ||
def rpcAskTimeout: RpcTimeout = | ||
new RpcTimeout(get(RPC_ASK_TIMEOUT).milli, RPC_ASK_TIMEOUT.key) | ||
def rpcTimeoutRetryWaitMs: Long = get(RPC_TIMEOUT_RETRY_WAIT) | ||
def rpcInMemoryBoundedInboxCapacity(): Int = { | ||
get(RPC_INBOX_CAPACITY) | ||
} | ||
|
@@ -901,6 +902,7 @@ class CelebornConf(loadDefaults: Boolean) extends Cloneable with Logging with Se | |
def clientCloseIdleConnections: Boolean = get(CLIENT_CLOSE_IDLE_CONNECTIONS) | ||
def clientRegisterShuffleMaxRetry: Int = get(CLIENT_REGISTER_SHUFFLE_MAX_RETRIES) | ||
def clientRegisterShuffleRetryWaitMs: Long = get(CLIENT_REGISTER_SHUFFLE_RETRY_WAIT) | ||
def clientCallLifecycleManagerMaxRetry: Int = get(CLIENT_CALL_LIFECYCLEMANAGER_MAX_RETRIES) | ||
def clientReserveSlotsRackAwareEnabled: Boolean = get(CLIENT_RESERVE_SLOTS_RACKAWARE_ENABLED) | ||
def clientReserveSlotsMaxRetries: Int = get(CLIENT_RESERVE_SLOTS_MAX_RETRIES) | ||
def clientReserveSlotsRetryWait: Long = get(CLIENT_RESERVE_SLOTS_RETRY_WAIT) | ||
|
@@ -4884,6 +4886,23 @@ object CelebornConf extends Logging { | |
.timeConf(TimeUnit.MILLISECONDS) | ||
.createWithDefaultString("3s") | ||
|
||
val RPC_TIMEOUT_RETRY_WAIT: ConfigEntry[Long] = | ||
buildConf("celeborn.rpc.timeoutRetryWait") | ||
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. Seems 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. has been updated~ |
||
.categories("network") | ||
.version("0.6.0") | ||
.doc("Wait time before next retry if RpcTimeoutException.") | ||
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. nit: |
||
.timeConf(TimeUnit.MILLISECONDS) | ||
.createWithDefaultString("1s") | ||
|
||
val CLIENT_CALL_LIFECYCLEMANAGER_MAX_RETRIES: ConfigEntry[Int] = | ||
buildConf("celeborn.client.callLifecycleManager.maxRetries") | ||
.withAlternative("celeborn.callLifecycleManager.maxRetries") | ||
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. Seems there is no legacy config 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. Is it possible to reuse CLIENT_RPC_MAX_RETIRES?
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. Too many parameters are not easy to maintain. 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. Maybe we can fallback the specific client config to a client default config item at least. |
||
.categories("client") | ||
.version("0.6.0") | ||
.doc("Max retry times for client to reserve slots.") | ||
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.
Seems not only for reserving slots. |
||
.intConf | ||
.createWithDefault(3) | ||
|
||
val CLIENT_RESERVE_SLOTS_MAX_RETRIES: ConfigEntry[Int] = | ||
buildConf("celeborn.client.reserveSlots.maxRetries") | ||
.withAlternative("celeborn.slots.reserve.maxRetries") | ||
|
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.
seems unnecessary change? I do not see new concurrent class involved in this class.