|
@@ -35,10 +35,10 @@ import java.net.URI;
|
|
|
* </p>
|
|
|
*
|
|
|
* <p>Applications specify the files, via urls (hdfs:// or http://) to be cached
|
|
|
- * via the {@link org.apache.hadoop.mapred.JobConf}.
|
|
|
- * The <code>DistributedCache</code> assumes that the
|
|
|
- * files specified via hdfs:// urls are already present on the
|
|
|
- * {@link FileSystem} at the path specified by the url.</p>
|
|
|
+ * via the {@link org.apache.hadoop.mapred.JobConf}. The
|
|
|
+ * <code>DistributedCache</code> assumes that the files specified via urls are
|
|
|
+ * already present on the {@link FileSystem} at the path specified by the url
|
|
|
+ * and are accessible by every machine in the cluster.</p>
|
|
|
*
|
|
|
* <p>The framework will copy the necessary files on to the slave node before
|
|
|
* any tasks for the job are executed on that node. Its efficiency stems from
|
|
@@ -129,9 +129,7 @@ public class DistributedCache {
|
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
|
*
|
|
|
* @param cache the cache to be localized, this should be specified as
|
|
|
- * new URI(hdfs://hostname:port/absolute_path_to_file#LINKNAME). If no schema
|
|
|
- * or hostname:port is provided the file is assumed to be in the filesystem
|
|
|
- * being used in the Configuration
|
|
|
+ * new URI(scheme://scheme-specific-part/absolute_path_to_file#LINKNAME).
|
|
|
* @param conf The Confguration file which contains the filesystem
|
|
|
* @param baseDir The base cache Dir where you wnat to localize the files/archives
|
|
|
* @param fileStatus The file status on the dfs.
|
|
@@ -162,9 +160,7 @@ public class DistributedCache {
|
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
|
*
|
|
|
* @param cache the cache to be localized, this should be specified as
|
|
|
- * new URI(hdfs://hostname:port/absolute_path_to_file#LINKNAME). If no schema
|
|
|
- * or hostname:port is provided the file is assumed to be in the filesystem
|
|
|
- * being used in the Configuration
|
|
|
+ * new URI(scheme://scheme-specific-part/absolute_path_to_file#LINKNAME).
|
|
|
* @param conf The Confguration file which contains the filesystem
|
|
|
* @param baseDir The base cache Dir where you wnat to localize the files/archives
|
|
|
* @param fileStatus The file status on the dfs.
|
|
@@ -231,9 +227,7 @@ public class DistributedCache {
|
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
|
*
|
|
|
* @param cache the cache to be localized, this should be specified as
|
|
|
- * new URI(hdfs://hostname:port/absolute_path_to_file#LINKNAME). If no schema
|
|
|
- * or hostname:port is provided the file is assumed to be in the filesystem
|
|
|
- * being used in the Configuration
|
|
|
+ * new URI(scheme://scheme-specific-part/absolute_path_to_file#LINKNAME).
|
|
|
* @param conf The Confguration file which contains the filesystem
|
|
|
* @param baseDir The base cache Dir where you wnat to localize the files/archives
|
|
|
* @param isArchive if the cache is an archive or a file. In case it is an
|
|
@@ -350,7 +344,7 @@ public class DistributedCache {
|
|
|
if(cache.getFragment() == null) {
|
|
|
doSymlink = false;
|
|
|
}
|
|
|
- FileSystem fs = getFileSystem(cache, conf);
|
|
|
+ FileSystem fs = FileSystem.get(cache, conf);
|
|
|
String link = currentWorkDir.toString() + Path.SEPARATOR + cache.getFragment();
|
|
|
File flink = new File(link);
|
|
|
if (ifExistsAndFresh(conf, fs, cache, confFileStamp,
|
|
@@ -533,14 +527,6 @@ public class DistributedCache {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static FileSystem getFileSystem(URI cache, Configuration conf)
|
|
|
- throws IOException {
|
|
|
- if ("hdfs".equals(cache.getScheme()))
|
|
|
- return FileSystem.get(cache, conf);
|
|
|
- else
|
|
|
- return FileSystem.get(conf);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Set the configuration with the given set of archives
|
|
|
* @param archives The list of archives that need to be localized
|