|
@@ -49,7 +49,8 @@ import org.apache.hadoop.mapreduce.task.reduce.MapOutput.Type;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
|
|
|
-@SuppressWarnings({"deprecation"})
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+
|
|
class Fetcher<K,V> extends Thread {
|
|
class Fetcher<K,V> extends Thread {
|
|
|
|
|
|
private static final Log LOG = LogFactory.getLog(Fetcher.class);
|
|
private static final Log LOG = LogFactory.getLog(Fetcher.class);
|
|
@@ -175,13 +176,18 @@ class Fetcher<K,V> extends Thread {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ protected HttpURLConnection openConnection(URL url) throws IOException {
|
|
|
|
+ return (HttpURLConnection)url.openConnection();
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* The crux of the matter...
|
|
* The crux of the matter...
|
|
*
|
|
*
|
|
* @param host {@link MapHost} from which we need to
|
|
* @param host {@link MapHost} from which we need to
|
|
* shuffle available map-outputs.
|
|
* shuffle available map-outputs.
|
|
*/
|
|
*/
|
|
- private void copyFromHost(MapHost host) throws IOException {
|
|
|
|
|
|
+ protected void copyFromHost(MapHost host) throws IOException {
|
|
// Get completed maps on 'host'
|
|
// Get completed maps on 'host'
|
|
List<TaskAttemptID> maps = scheduler.getMapsForHost(host);
|
|
List<TaskAttemptID> maps = scheduler.getMapsForHost(host);
|
|
|
|
|
|
@@ -191,9 +197,11 @@ class Fetcher<K,V> extends Thread {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
- LOG.debug("Fetcher " + id + " going to fetch from " + host);
|
|
|
|
- for (TaskAttemptID tmp: maps) {
|
|
|
|
- LOG.debug(tmp);
|
|
|
|
|
|
+ if(LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Fetcher " + id + " going to fetch from " + host);
|
|
|
|
+ for (TaskAttemptID tmp: maps) {
|
|
|
|
+ LOG.debug(tmp);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
// List of maps to be fetched yet
|
|
// List of maps to be fetched yet
|
|
@@ -205,7 +213,7 @@ class Fetcher<K,V> extends Thread {
|
|
|
|
|
|
try {
|
|
try {
|
|
URL url = getMapOutputURL(host, maps);
|
|
URL url = getMapOutputURL(host, maps);
|
|
- HttpURLConnection connection = (HttpURLConnection)url.openConnection();
|
|
|
|
|
|
+ HttpURLConnection connection = openConnection(url);
|
|
|
|
|
|
// generate hash of the url
|
|
// generate hash of the url
|
|
String msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
|
|
String msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
|
|
@@ -266,17 +274,24 @@ class Fetcher<K,V> extends Thread {
|
|
|
|
|
|
try {
|
|
try {
|
|
// Loop through available map-outputs and fetch them
|
|
// Loop through available map-outputs and fetch them
|
|
- // On any error, good becomes false and we exit after putting back
|
|
|
|
- // the remaining maps to the yet_to_be_fetched list
|
|
|
|
- boolean good = true;
|
|
|
|
- while (!remaining.isEmpty() && good) {
|
|
|
|
- good = copyMapOutput(host, input, remaining);
|
|
|
|
|
|
+ // On any error, faildTasks is not null and we exit
|
|
|
|
+ // after putting back the remaining maps to the
|
|
|
|
+ // yet_to_be_fetched list and marking the failed tasks.
|
|
|
|
+ TaskAttemptID[] failedTasks = null;
|
|
|
|
+ while (!remaining.isEmpty() && failedTasks == null) {
|
|
|
|
+ failedTasks = copyMapOutput(host, input, remaining);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if(failedTasks != null) {
|
|
|
|
+ for(TaskAttemptID left: failedTasks) {
|
|
|
|
+ scheduler.copyFailed(left, host, true);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
IOUtils.cleanup(LOG, input);
|
|
IOUtils.cleanup(LOG, input);
|
|
|
|
|
|
// Sanity check
|
|
// Sanity check
|
|
- if (good && !remaining.isEmpty()) {
|
|
|
|
|
|
+ if (failedTasks == null && !remaining.isEmpty()) {
|
|
throw new IOException("server didn't return all expected map outputs: "
|
|
throw new IOException("server didn't return all expected map outputs: "
|
|
+ remaining.size() + " left.");
|
|
+ remaining.size() + " left.");
|
|
}
|
|
}
|
|
@@ -285,10 +300,9 @@ class Fetcher<K,V> extends Thread {
|
|
scheduler.putBackKnownMapOutput(host, left);
|
|
scheduler.putBackKnownMapOutput(host, left);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- }
|
|
|
|
|
|
+ }
|
|
|
|
|
|
- private boolean copyMapOutput(MapHost host,
|
|
|
|
|
|
+ private TaskAttemptID[] copyMapOutput(MapHost host,
|
|
DataInputStream input,
|
|
DataInputStream input,
|
|
Set<TaskAttemptID> remaining) {
|
|
Set<TaskAttemptID> remaining) {
|
|
MapOutput<K,V> mapOutput = null;
|
|
MapOutput<K,V> mapOutput = null;
|
|
@@ -310,14 +324,15 @@ class Fetcher<K,V> extends Thread {
|
|
} catch (IllegalArgumentException e) {
|
|
} catch (IllegalArgumentException e) {
|
|
badIdErrs.increment(1);
|
|
badIdErrs.increment(1);
|
|
LOG.warn("Invalid map id ", e);
|
|
LOG.warn("Invalid map id ", e);
|
|
- return false;
|
|
|
|
|
|
+ //Don't know which one was bad, so consider all of them as bad
|
|
|
|
+ return remaining.toArray(new TaskAttemptID[remaining.size()]);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
// Do some basic sanity verification
|
|
// Do some basic sanity verification
|
|
if (!verifySanity(compressedLength, decompressedLength, forReduce,
|
|
if (!verifySanity(compressedLength, decompressedLength, forReduce,
|
|
remaining, mapId)) {
|
|
remaining, mapId)) {
|
|
- return false;
|
|
|
|
|
|
+ return new TaskAttemptID[] {mapId};
|
|
}
|
|
}
|
|
|
|
|
|
LOG.debug("header: " + mapId + ", len: " + compressedLength +
|
|
LOG.debug("header: " + mapId + ", len: " + compressedLength +
|
|
@@ -329,7 +344,7 @@ class Fetcher<K,V> extends Thread {
|
|
// Check if we can shuffle *now* ...
|
|
// Check if we can shuffle *now* ...
|
|
if (mapOutput.getType() == Type.WAIT) {
|
|
if (mapOutput.getType() == Type.WAIT) {
|
|
LOG.info("fetcher#" + id + " - MergerManager returned Status.WAIT ...");
|
|
LOG.info("fetcher#" + id + " - MergerManager returned Status.WAIT ...");
|
|
- return false;
|
|
|
|
|
|
+ return new TaskAttemptID[] {mapId};
|
|
}
|
|
}
|
|
|
|
|
|
// Go!
|
|
// Go!
|
|
@@ -351,14 +366,18 @@ class Fetcher<K,V> extends Thread {
|
|
// Note successful shuffle
|
|
// Note successful shuffle
|
|
remaining.remove(mapId);
|
|
remaining.remove(mapId);
|
|
metrics.successFetch();
|
|
metrics.successFetch();
|
|
- return true;
|
|
|
|
|
|
+ return null;
|
|
} catch (IOException ioe) {
|
|
} catch (IOException ioe) {
|
|
ioErrs.increment(1);
|
|
ioErrs.increment(1);
|
|
if (mapId == null || mapOutput == null) {
|
|
if (mapId == null || mapOutput == null) {
|
|
LOG.info("fetcher#" + id + " failed to read map header" +
|
|
LOG.info("fetcher#" + id + " failed to read map header" +
|
|
mapId + " decomp: " +
|
|
mapId + " decomp: " +
|
|
decompressedLength + ", " + compressedLength, ioe);
|
|
decompressedLength + ", " + compressedLength, ioe);
|
|
- return false;
|
|
|
|
|
|
+ if(mapId == null) {
|
|
|
|
+ return remaining.toArray(new TaskAttemptID[remaining.size()]);
|
|
|
|
+ } else {
|
|
|
|
+ return new TaskAttemptID[] {mapId};
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
LOG.info("Failed to shuffle output of " + mapId +
|
|
LOG.info("Failed to shuffle output of " + mapId +
|
|
@@ -366,9 +385,8 @@ class Fetcher<K,V> extends Thread {
|
|
|
|
|
|
// Inform the shuffle-scheduler
|
|
// Inform the shuffle-scheduler
|
|
mapOutput.abort();
|
|
mapOutput.abort();
|
|
- scheduler.copyFailed(mapId, host, true);
|
|
|
|
metrics.failedFetch();
|
|
metrics.failedFetch();
|
|
- return false;
|
|
|
|
|
|
+ return new TaskAttemptID[] {mapId};
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|