MAPREDUCE-5601. ShuffleHandler fadvises file regions as DONTNEED even when fetch fails (Sandy Ryza)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1537856 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ba85796267
commit
d023d52062
|
@ -34,6 +34,9 @@ Release 2.3.0 - UNRELEASED
|
||||||
MAPREDUCE-5487. In task processes, JobConf is unnecessarily loaded again
|
MAPREDUCE-5487. In task processes, JobConf is unnecessarily loaded again
|
||||||
in Limits (Sandy Ryza)
|
in Limits (Sandy Ryza)
|
||||||
|
|
||||||
|
MAPREDUCE-5601. ShuffleHandler fadvises file regions as DONTNEED even when
|
||||||
|
fetch fails (Sandy Ryza)
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
||||||
MAPREDUCE-5316. job -list-attempt-ids command does not handle illegal
|
MAPREDUCE-5316. job -list-attempt-ids command does not handle illegal
|
||||||
|
|
|
@ -69,6 +69,14 @@ public class FadvisedFileRegion extends DefaultFileRegion {
|
||||||
if (readaheadRequest != null) {
|
if (readaheadRequest != null) {
|
||||||
readaheadRequest.cancel();
|
readaheadRequest.cancel();
|
||||||
}
|
}
|
||||||
|
super.releaseExternalResources();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Call when the transfer completes successfully so we can advise the OS that
|
||||||
|
* we don't need the region to be cached anymore.
|
||||||
|
*/
|
||||||
|
public void transferSuccessful() {
|
||||||
if (manageOsCache && getCount() > 0) {
|
if (manageOsCache && getCount() > 0) {
|
||||||
try {
|
try {
|
||||||
NativeIO.POSIX.posixFadviseIfPossible(identifier,
|
NativeIO.POSIX.posixFadviseIfPossible(identifier,
|
||||||
|
@ -78,6 +86,5 @@ public class FadvisedFileRegion extends DefaultFileRegion {
|
||||||
LOG.warn("Failed to manage OS cache for " + identifier, t);
|
LOG.warn("Failed to manage OS cache for " + identifier, t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
super.releaseExternalResources();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -625,6 +625,9 @@ public class ShuffleHandler extends AuxiliaryService {
|
||||||
// attribute to appropriate spill output
|
// attribute to appropriate spill output
|
||||||
@Override
|
@Override
|
||||||
public void operationComplete(ChannelFuture future) {
|
public void operationComplete(ChannelFuture future) {
|
||||||
|
if (future.isSuccess()) {
|
||||||
|
partition.transferSuccessful();
|
||||||
|
}
|
||||||
partition.releaseExternalResources();
|
partition.releaseExternalResources();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
Loading…
Reference in New Issue