jerqi commented on code in PR #55:
URL: https://github.com/apache/incubator-uniffle/pull/55#discussion_r921211301


##########
client-mr/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java:
##########
@@ -199,6 +200,24 @@ public Thread newThread(Runnable r) {
       extraConf.set(RssMRConfig.RSS_REMOTE_STORAGE_PATH, 
remoteStorage.getPath());
       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,
+        RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED_DEFAULT)) {
+
+        if (storageType != StorageType.MEMORY_LOCALFILE_HDFS.name()) {
+          throw new IllegalArgumentException("Remote spill only supports "
+            + StorageType.MEMORY_LOCALFILE_HDFS.name() + " mode");
+        }
+
+        // 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);

Review Comment:
   It's odd.  The logic is wrong.



##########
client-mr/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java:
##########
@@ -199,6 +200,24 @@ public Thread newThread(Runnable r) {
       extraConf.set(RssMRConfig.RSS_REMOTE_STORAGE_PATH, 
remoteStorage.getPath());
       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,
+        RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED_DEFAULT)) {
+
+        if (storageType != StorageType.MEMORY_LOCALFILE_HDFS.name()) {
+          throw new IllegalArgumentException("Remote spill only supports "
+            + StorageType.MEMORY_LOCALFILE_HDFS.name() + " mode");
+        }
+
+        // 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);

Review Comment:
   We shouldn't modify the configuration. It will affect the replication of 
output.



##########
client-mr/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java:
##########
@@ -199,6 +200,24 @@ public Thread newThread(Runnable r) {
       extraConf.set(RssMRConfig.RSS_REMOTE_STORAGE_PATH, 
remoteStorage.getPath());
       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,
+        RssMRConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED_DEFAULT)) {
+
+        if (storageType != StorageType.MEMORY_LOCALFILE_HDFS.name()) {

Review Comment:
   Could we check check remote storage path instead?



-- 
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]

Reply via email to