waitinfuture commented on code in PR #1002:
URL:
https://github.com/apache/incubator-celeborn/pull/1002#discussion_r1036117279
##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1205,4 +1203,233 @@ private boolean connectFail(String message) {
|| (message.equals("Connection reset by peer"))
|| (message.startsWith("Failed to send RPC "));
}
+
+ @Override
+ public void pushDataHandShake(
+ String applicationId,
+ int shuffleId,
+ int mapId,
+ int attemptId,
+ int numPartitions,
+ int bufferSize,
+ PartitionLocation location)
+ throws IOException {
+ sendMessageInternal(
+ shuffleId,
+ mapId,
+ attemptId,
+ location,
+ () -> {
+ String shuffleKey = Utils.makeShuffleKey(applicationId, shuffleId);
+ logger.info(
+ "pushDataHandShake shuffleKey:{}, attemptId:{}, locationId:{}",
+ shuffleKey,
+ attemptId,
+ location.getUniqueId());
+ logger.debug("pushDataHandShake location:{}", location.toString());
Review Comment:
useless toString()
##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1205,4 +1203,233 @@ private boolean connectFail(String message) {
|| (message.equals("Connection reset by peer"))
|| (message.startsWith("Failed to send RPC "));
}
+
+ @Override
+ public void pushDataHandShake(
+ String applicationId,
+ int shuffleId,
+ int mapId,
+ int attemptId,
+ int numPartitions,
+ int bufferSize,
+ PartitionLocation location)
+ throws IOException {
+ sendMessageInternal(
+ shuffleId,
+ mapId,
+ attemptId,
+ location,
+ () -> {
+ String shuffleKey = Utils.makeShuffleKey(applicationId, shuffleId);
+ logger.info(
+ "pushDataHandShake shuffleKey:{}, attemptId:{}, locationId:{}",
+ shuffleKey,
+ attemptId,
+ location.getUniqueId());
+ logger.debug("pushDataHandShake location:{}", location.toString());
+ TransportClient client =
+ dataClientFactory.createClient(location.getHost(),
location.getPushPort());
+ PushDataHandShake handShake =
+ new PushDataHandShake(
+ MASTER_MODE,
+ shuffleKey,
+ location.getUniqueId(),
+ attemptId,
+ numPartitions,
+ bufferSize);
+ client.sendRpcSync(handShake.toByteBuffer(),
conf.pushDataRpcTimeoutMs());
+ return null;
+ });
+ }
+
+ @Override
+ public Optional<PartitionLocation> regionStart(
+ String applicationId,
+ int shuffleId,
+ int mapId,
+ int attemptId,
+ PartitionLocation location,
+ int currentRegionIdx,
+ boolean isBroadcast)
+ throws IOException {
+ return sendMessageInternal(
+ shuffleId,
+ mapId,
+ attemptId,
+ location,
+ () -> {
+ String shuffleKey = Utils.makeShuffleKey(applicationId, shuffleId);
+ logger.info(
+ "regionStart shuffleKey:{}, attemptId:{}, locationId:{}",
+ shuffleKey,
+ attemptId,
+ location.getUniqueId());
+ logger.debug("regionStart location:{}", location.toString());
+ TransportClient client =
+ dataClientFactory.createClient(location.getHost(),
location.getPushPort());
+ RegionStart regionStart =
+ new RegionStart(
+ MASTER_MODE,
+ shuffleKey,
+ location.getUniqueId(),
+ attemptId,
+ currentRegionIdx,
+ isBroadcast);
+ ByteBuffer regionStartResponse =
+ client.sendRpcSync(regionStart.toByteBuffer(),
conf.pushDataRpcTimeoutMs());
+ if (regionStartResponse.hasRemaining()
+ && regionStartResponse.get() ==
StatusCode.HARD_SPLIT.getValue()) {
+ // if split then revive
+ PbChangeLocationResponse response =
+ driverRssMetaService.askSync(
+ ControlMessages.Revive$.MODULE$.apply(
+ applicationId,
+ shuffleId,
+ mapId,
+ attemptId,
+ location.getId(),
+ location.getEpoch(),
+ location,
+ StatusCode.HARD_SPLIT),
+ ClassTag$.MODULE$.apply(PbChangeLocationResponse.class));
+ // per partitionKey only serve single PartitionLocation in Client
Cache.
+ StatusCode respStatus = Utils.toStatusCode(response.getStatus());
+ if (StatusCode.SUCCESS.equals(respStatus)) {
+ return
Optional.of(PbSerDeUtils.fromPbPartitionLocation(response.getLocation()));
+ } else if (StatusCode.MAP_ENDED.equals(respStatus)) {
+ final String mapKey = Utils.makeMapKey(shuffleId, mapId,
attemptId);
+ mapperEndMap
+ .computeIfAbsent(shuffleId, (id) ->
ConcurrentHashMap.newKeySet())
+ .add(mapKey);
+ return Optional.empty();
+ } else {
+ // throw exception
+ logger.error(
+ "Exception raised while reviving for shuffle {} reduce {}
epoch {}.",
+ shuffleId,
+ location.getId(),
+ location.getEpoch());
+ throw new IOException("regiontstart revive failed");
+ }
+ }
+ return Optional.empty();
+ });
+ }
+
+ @Override
+ public void regionFinish(
+ String applicationId, int shuffleId, int mapId, int attemptId,
PartitionLocation location)
+ throws IOException {
+ sendMessageInternal(
+ shuffleId,
+ mapId,
+ attemptId,
+ location,
+ () -> {
+ final String shuffleKey = Utils.makeShuffleKey(applicationId,
shuffleId);
+ logger.info(
+ "regionFinish shuffleKey:{}, attemptId:{}, locationId:{}",
+ shuffleKey,
+ attemptId,
+ location.getUniqueId());
+ logger.debug("regionFinish location:{}", location.toString());
+ TransportClient client =
+ dataClientFactory.createClient(location.getHost(),
location.getPushPort());
+ RegionFinish regionFinish =
+ new RegionFinish(MASTER_MODE, shuffleKey,
location.getUniqueId(), attemptId);
+ client.sendRpcSync(regionFinish.toByteBuffer(),
conf.pushDataRpcTimeoutMs());
+ return null;
+ });
+ }
+
+ private <R> R sendMessageInternal(
+ int shuffleId,
+ int mapId,
+ int attemptId,
+ PartitionLocation location,
+ ThrowingExceptionSupplier<R, Exception> supplier)
+ throws IOException {
+ PushState pushState = null;
+ int batchId = 0;
+ try {
+ // mapKey
+ final String mapKey = Utils.makeMapKey(shuffleId, mapId, attemptId);
+ // return if shuffle stage already ended
+ if (mapperEnded(shuffleId, mapId, attemptId)) {
+ logger.debug(
+ "The mapper(shuffle {} map {} attempt {}) has already ended while"
+ " pushing data.",
+ shuffleId,
+ mapId,
+ attemptId);
+ return null;
+ }
+ pushState = pushStates.computeIfAbsent(mapKey, (s) -> new
PushState(conf));
+ // check limit
+ limitMaxInFlight(mapKey, pushState, maxInFlight);
Review Comment:
should we limit zero here?
##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1205,4 +1203,233 @@ private boolean connectFail(String message) {
|| (message.equals("Connection reset by peer"))
|| (message.startsWith("Failed to send RPC "));
}
+
+ @Override
+ public void pushDataHandShake(
+ String applicationId,
+ int shuffleId,
+ int mapId,
+ int attemptId,
+ int numPartitions,
+ int bufferSize,
+ PartitionLocation location)
+ throws IOException {
+ sendMessageInternal(
+ shuffleId,
+ mapId,
+ attemptId,
+ location,
+ () -> {
+ String shuffleKey = Utils.makeShuffleKey(applicationId, shuffleId);
+ logger.info(
+ "pushDataHandShake shuffleKey:{}, attemptId:{}, locationId:{}",
+ shuffleKey,
+ attemptId,
+ location.getUniqueId());
+ logger.debug("pushDataHandShake location:{}", location.toString());
+ TransportClient client =
+ dataClientFactory.createClient(location.getHost(),
location.getPushPort());
+ PushDataHandShake handShake =
+ new PushDataHandShake(
+ MASTER_MODE,
+ shuffleKey,
+ location.getUniqueId(),
+ attemptId,
+ numPartitions,
+ bufferSize);
+ client.sendRpcSync(handShake.toByteBuffer(),
conf.pushDataRpcTimeoutMs());
+ return null;
+ });
+ }
+
+ @Override
+ public Optional<PartitionLocation> regionStart(
+ String applicationId,
+ int shuffleId,
+ int mapId,
+ int attemptId,
+ PartitionLocation location,
+ int currentRegionIdx,
+ boolean isBroadcast)
+ throws IOException {
+ return sendMessageInternal(
+ shuffleId,
+ mapId,
+ attemptId,
+ location,
+ () -> {
+ String shuffleKey = Utils.makeShuffleKey(applicationId, shuffleId);
+ logger.info(
+ "regionStart shuffleKey:{}, attemptId:{}, locationId:{}",
+ shuffleKey,
+ attemptId,
+ location.getUniqueId());
+ logger.debug("regionStart location:{}", location.toString());
Review Comment:
ditto
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]