MAPREDUCE-5355. MiniMRYarnCluster with localFs does not work on Windows. Contributed by Chuan Liu.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1499148 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a52d85834d
commit
f047f9cac6
@ -600,6 +600,9 @@ Release 2.1.0-beta - 2013-07-02
|
|||||||
MAPREDUCE-5177. Use common utils FileUtil#setReadable/Writable/Executable &
|
MAPREDUCE-5177. Use common utils FileUtil#setReadable/Writable/Executable &
|
||||||
FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
|
FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
|
||||||
|
|
||||||
|
MAPREDUCE-5355. MiniMRYarnCluster with localFs does not work on Windows.
|
||||||
|
(Chuan Liu via cnauroth)
|
||||||
|
|
||||||
MAPREDUCE-5291. Change MR App to use updated property names in
|
MAPREDUCE-5291. Change MR App to use updated property names in
|
||||||
container-log4j.properties. (Zhijie Shen via sseth)
|
container-log4j.properties. (Zhijie Shen via sseth)
|
||||||
|
|
||||||
|
@ -26,6 +26,7 @@
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
import org.apache.hadoop.fs.FileContext;
|
import org.apache.hadoop.fs.FileContext;
|
||||||
|
import org.apache.hadoop.fs.LocalFileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.mapred.LocalContainerLauncher;
|
import org.apache.hadoop.mapred.LocalContainerLauncher;
|
||||||
import org.apache.hadoop.mapred.ShuffleHandler;
|
import org.apache.hadoop.mapred.ShuffleHandler;
|
||||||
@ -87,6 +88,21 @@ public void serviceInit(Configuration conf) throws Exception {
|
|||||||
try {
|
try {
|
||||||
Path stagingPath = FileContext.getFileContext(conf).makeQualified(
|
Path stagingPath = FileContext.getFileContext(conf).makeQualified(
|
||||||
new Path(conf.get(MRJobConfig.MR_AM_STAGING_DIR)));
|
new Path(conf.get(MRJobConfig.MR_AM_STAGING_DIR)));
|
||||||
|
/*
|
||||||
|
* Re-configure the staging path on Windows if the file system is localFs.
|
||||||
|
* We need to use a absolute path that contains the drive letter. The unit
|
||||||
|
* test could run on a different drive than the AM. We can run into the
|
||||||
|
* issue that job files are localized to the drive where the test runs on,
|
||||||
|
* while the AM starts on a different drive and fails to find the job
|
||||||
|
* metafiles. Using absolute path can avoid this ambiguity.
|
||||||
|
*/
|
||||||
|
if (Path.WINDOWS) {
|
||||||
|
if (LocalFileSystem.class.isInstance(stagingPath.getFileSystem(conf))) {
|
||||||
|
conf.set(MRJobConfig.MR_AM_STAGING_DIR,
|
||||||
|
new File(conf.get(MRJobConfig.MR_AM_STAGING_DIR))
|
||||||
|
.getAbsolutePath());
|
||||||
|
}
|
||||||
|
}
|
||||||
FileContext fc=FileContext.getFileContext(stagingPath.toUri(), conf);
|
FileContext fc=FileContext.getFileContext(stagingPath.toUri(), conf);
|
||||||
if (fc.util().exists(stagingPath)) {
|
if (fc.util().exists(stagingPath)) {
|
||||||
LOG.info(stagingPath + " exists! deleting...");
|
LOG.info(stagingPath + " exists! deleting...");
|
||||||
|
Loading…
Reference in New Issue
Block a user