jerqi commented on code in PR #55:
URL: https://github.com/apache/incubator-uniffle/pull/55#discussion_r921861163
##########
client-mr/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java:
##########
@@ -201,21 +201,22 @@ public Thread newThread(Runnable r) {
extraConf.set(RssMRConfig.RSS_REMOTE_STORAGE_CONF,
remoteStorage.getConfString());
// When containers have disk with very limited space, reduce is allowed
to spill data to hdfs
- if (extraConf.getBoolean(RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED,
+ if (conf.getBoolean(RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED,
RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED_DEFAULT)) {
- if (storageType != StorageType.MEMORY_LOCALFILE_HDFS.name()) {
+ if (storageType != StorageType.MEMORY_LOCALFILE_HDFS.name()
+ || remoteStorage.isEmpty()) {
throw new IllegalArgumentException("Remote spill only supports "
- + StorageType.MEMORY_LOCALFILE_HDFS.name() + " mode");
+ + StorageType.MEMORY_LOCALFILE_HDFS.name() + " mode with " +
remoteStorage);
}
- // Use minimized replica, because spilled data can be recomputed by
reduce task.
- // And task re-computation is much cheaper job re-computation
- conf.setInt("dfs.replication", 1);
// When remote spill is enabled, reduce task is more easy to crash.
// We allow more attempts to avoid recomputing job.
- conf.setInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 5);
+ int originalAttempts = conf.getInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 4);
+ int inc = conf.getInt(RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ATTEMPT_INC,
Review Comment:
checkValue >= 0
##########
client-mr/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java:
##########
@@ -201,21 +201,22 @@ public Thread newThread(Runnable r) {
extraConf.set(RssMRConfig.RSS_REMOTE_STORAGE_CONF,
remoteStorage.getConfString());
// When containers have disk with very limited space, reduce is allowed
to spill data to hdfs
- if (extraConf.getBoolean(RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED,
+ if (conf.getBoolean(RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED,
RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED_DEFAULT)) {
- if (storageType != StorageType.MEMORY_LOCALFILE_HDFS.name()) {
+ if (storageType != StorageType.MEMORY_LOCALFILE_HDFS.name()
+ || remoteStorage.isEmpty()) {
Review Comment:
```
if (remoteStorage.isEmpty())
```
we can look at the fetchRemoteStorage internal implement.
##########
client-mr/src/main/java/org/apache/hadoop/mapreduce/task/reduce/RssRemoteMergeManagerImpl.java:
##########
@@ -140,6 +141,14 @@ public RssRemoteMergeManagerImpl(String appId,
TaskAttemptID reduceId, JobConf j
this.mergedMapOutputsCounter = mergedMapOutputsCounter;
try {
+ // Use minimized replica, because spilled data can be recomputed by
reduce task.
+ // Instead, we use more retries on HDFS client.
+ int replication =
remoteConf.getInt(RssMRConfig.RSS_REDUCE_REMOTE_SPILL_REPLICATION,
Review Comment:
There seems to be logic error.
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]