|
@@ -18,12 +18,12 @@
|
|
|
|
|
|
package org.apache.hadoop.filecache;
|
|
package org.apache.hadoop.filecache;
|
|
|
|
|
|
-import org.apache.commons.logging.*;
|
|
|
|
import java.io.*;
|
|
import java.io.*;
|
|
import java.util.*;
|
|
import java.util.*;
|
|
import org.apache.hadoop.conf.*;
|
|
import org.apache.hadoop.conf.*;
|
|
import org.apache.hadoop.util.*;
|
|
import org.apache.hadoop.util.*;
|
|
import org.apache.hadoop.fs.*;
|
|
import org.apache.hadoop.fs.*;
|
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
|
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
|
|
|
|
@@ -108,22 +108,23 @@ import java.net.URI;
|
|
* }
|
|
* }
|
|
*
|
|
*
|
|
* </pre></blockquote></p>
|
|
* </pre></blockquote></p>
|
|
- *
|
|
|
|
|
|
+ * It is also very common to use the DistributedCache by using
|
|
|
|
+ * {@link org.apache.hadoop.util.GenericOptionsParser}.
|
|
|
|
+ *
|
|
|
|
+ * This class includes methods that should be used by users
|
|
|
|
+ * (specifically those mentioned in the example above, as well
|
|
|
|
+ * as {@link DistributedCache#addArchiveToClassPath(Path, Configuration)}),
|
|
|
|
+ * as well as methods intended for use by the MapReduce framework
|
|
|
|
+ * (e.g., {@link org.apache.hadoop.mapred.JobClient}). For implementation
|
|
|
|
+ * details, see {@link TrackerDistributedCacheManager} and
|
|
|
|
+ * {@link TaskDistributedCacheManager}.
|
|
|
|
+ *
|
|
|
|
+ * @see TrackerDistributedCacheManager
|
|
|
|
+ * @see TaskDistributedCacheManager
|
|
* @see org.apache.hadoop.mapred.JobConf
|
|
* @see org.apache.hadoop.mapred.JobConf
|
|
* @see org.apache.hadoop.mapred.JobClient
|
|
* @see org.apache.hadoop.mapred.JobClient
|
|
*/
|
|
*/
|
|
public class DistributedCache {
|
|
public class DistributedCache {
|
|
- // cacheID to cacheStatus mapping
|
|
|
|
- private static TreeMap<String, CacheStatus> cachedArchives = new TreeMap<String, CacheStatus>();
|
|
|
|
-
|
|
|
|
- private static TreeMap<Path, Long> baseDirSize = new TreeMap<Path, Long>();
|
|
|
|
-
|
|
|
|
- // default total cache size
|
|
|
|
- private static final long DEFAULT_CACHE_SIZE = 10737418240L;
|
|
|
|
-
|
|
|
|
- private static final Log LOG =
|
|
|
|
- LogFactory.getLog(DistributedCache.class);
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Get the locally cached file or archive; it could either be
|
|
* Get the locally cached file or archive; it could either be
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
@@ -148,15 +149,18 @@ public class DistributedCache {
|
|
* @return the path to directory where the archives are unjarred in case of archives,
|
|
* @return the path to directory where the archives are unjarred in case of archives,
|
|
* the path to the file where the file is copied locally
|
|
* the path to the file where the file is copied locally
|
|
* @throws IOException
|
|
* @throws IOException
|
|
|
|
+ * @deprecated Internal to MapReduce framework. Use DistributedCacheManager
|
|
|
|
+ * instead.
|
|
*/
|
|
*/
|
|
public static Path getLocalCache(URI cache, Configuration conf,
|
|
public static Path getLocalCache(URI cache, Configuration conf,
|
|
Path baseDir, FileStatus fileStatus,
|
|
Path baseDir, FileStatus fileStatus,
|
|
boolean isArchive, long confFileStamp,
|
|
boolean isArchive, long confFileStamp,
|
|
Path currentWorkDir)
|
|
Path currentWorkDir)
|
|
- throws IOException {
|
|
|
|
|
|
+ throws IOException {
|
|
return getLocalCache(cache, conf, baseDir, fileStatus, isArchive,
|
|
return getLocalCache(cache, conf, baseDir, fileStatus, isArchive,
|
|
confFileStamp, currentWorkDir, true);
|
|
confFileStamp, currentWorkDir, true);
|
|
}
|
|
}
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Get the locally cached file or archive; it could either be
|
|
* Get the locally cached file or archive; it could either be
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
@@ -184,48 +188,19 @@ public class DistributedCache {
|
|
* @return the path to directory where the archives are unjarred in case of archives,
|
|
* @return the path to directory where the archives are unjarred in case of archives,
|
|
* the path to the file where the file is copied locally
|
|
* the path to the file where the file is copied locally
|
|
* @throws IOException
|
|
* @throws IOException
|
|
|
|
+ * @deprecated Internal to MapReduce framework. Use DistributedCacheManager
|
|
|
|
+ * instead.
|
|
*/
|
|
*/
|
|
public static Path getLocalCache(URI cache, Configuration conf,
|
|
public static Path getLocalCache(URI cache, Configuration conf,
|
|
Path baseDir, FileStatus fileStatus,
|
|
Path baseDir, FileStatus fileStatus,
|
|
boolean isArchive, long confFileStamp,
|
|
boolean isArchive, long confFileStamp,
|
|
- Path currentWorkDir, boolean honorSymLinkConf)
|
|
|
|
- throws IOException {
|
|
|
|
- String cacheId = makeRelative(cache, conf);
|
|
|
|
- CacheStatus lcacheStatus;
|
|
|
|
- Path localizedPath;
|
|
|
|
- synchronized (cachedArchives) {
|
|
|
|
- lcacheStatus = cachedArchives.get(cacheId);
|
|
|
|
- if (lcacheStatus == null) {
|
|
|
|
- // was never localized
|
|
|
|
- lcacheStatus = new CacheStatus(baseDir, new Path(baseDir, new Path(cacheId)));
|
|
|
|
- cachedArchives.put(cacheId, lcacheStatus);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- synchronized (lcacheStatus) {
|
|
|
|
- localizedPath = localizeCache(conf, cache, confFileStamp, lcacheStatus,
|
|
|
|
- fileStatus, isArchive, currentWorkDir, honorSymLinkConf);
|
|
|
|
- lcacheStatus.refcount++;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ Path currentWorkDir, boolean honorSymLinkConf) throws IOException {
|
|
|
|
|
|
- // try deleting stuff if you can
|
|
|
|
- long size = 0;
|
|
|
|
- synchronized (baseDirSize) {
|
|
|
|
- Long get = baseDirSize.get(baseDir);
|
|
|
|
- if ( get != null ) {
|
|
|
|
- size = get.longValue();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- // setting the cache size to a default of 10GB
|
|
|
|
- long allowedSize = conf.getLong("local.cache.size", DEFAULT_CACHE_SIZE);
|
|
|
|
- if (allowedSize < size) {
|
|
|
|
- // try some cache deletions
|
|
|
|
- deleteCache(conf);
|
|
|
|
- }
|
|
|
|
- return localizedPath;
|
|
|
|
|
|
+ return new TrackerDistributedCacheManager(conf).getLocalCache(cache, conf,
|
|
|
|
+ baseDir, fileStatus, isArchive, confFileStamp, currentWorkDir,
|
|
|
|
+ honorSymLinkConf);
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Get the locally cached file or archive; it could either be
|
|
* Get the locally cached file or archive; it could either be
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
* previously cached (and valid) or copy it from the {@link FileSystem} now.
|
|
@@ -249,17 +224,18 @@ public class DistributedCache {
|
|
* @return the path to directory where the archives are unjarred in case of archives,
|
|
* @return the path to directory where the archives are unjarred in case of archives,
|
|
* the path to the file where the file is copied locally
|
|
* the path to the file where the file is copied locally
|
|
* @throws IOException
|
|
* @throws IOException
|
|
-
|
|
|
|
|
|
+ * @deprecated Internal to MapReduce framework. Use DistributedCacheManager
|
|
|
|
+ * instead.
|
|
*/
|
|
*/
|
|
public static Path getLocalCache(URI cache, Configuration conf,
|
|
public static Path getLocalCache(URI cache, Configuration conf,
|
|
Path baseDir, boolean isArchive,
|
|
Path baseDir, boolean isArchive,
|
|
long confFileStamp, Path currentWorkDir)
|
|
long confFileStamp, Path currentWorkDir)
|
|
- throws IOException {
|
|
|
|
|
|
+ throws IOException {
|
|
return getLocalCache(cache, conf,
|
|
return getLocalCache(cache, conf,
|
|
baseDir, null, isArchive,
|
|
baseDir, null, isArchive,
|
|
confFileStamp, currentWorkDir);
|
|
confFileStamp, currentWorkDir);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* This is the opposite of getlocalcache. When you are done with
|
|
* This is the opposite of getlocalcache. When you are done with
|
|
* using the cache, you need to release the cache
|
|
* using the cache, you need to release the cache
|
|
@@ -267,232 +243,28 @@ public class DistributedCache {
|
|
* @param conf configuration which contains the filesystem the cache
|
|
* @param conf configuration which contains the filesystem the cache
|
|
* is contained in.
|
|
* is contained in.
|
|
* @throws IOException
|
|
* @throws IOException
|
|
|
|
+ * @deprecated Internal to MapReduce framework. Use DistributedCacheManager
|
|
|
|
+ * instead.
|
|
*/
|
|
*/
|
|
public static void releaseCache(URI cache, Configuration conf)
|
|
public static void releaseCache(URI cache, Configuration conf)
|
|
- throws IOException {
|
|
|
|
- String cacheId = makeRelative(cache, conf);
|
|
|
|
- synchronized (cachedArchives) {
|
|
|
|
- CacheStatus lcacheStatus = cachedArchives.get(cacheId);
|
|
|
|
- if (lcacheStatus == null)
|
|
|
|
- return;
|
|
|
|
- synchronized (lcacheStatus) {
|
|
|
|
- lcacheStatus.refcount--;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ throws IOException {
|
|
|
|
+ new TrackerDistributedCacheManager(conf).releaseCache(cache, conf);
|
|
}
|
|
}
|
|
|
|
|
|
- // To delete the caches which have a refcount of zero
|
|
|
|
-
|
|
|
|
- private static void deleteCache(Configuration conf) throws IOException {
|
|
|
|
- // try deleting cache Status with refcount of zero
|
|
|
|
- synchronized (cachedArchives) {
|
|
|
|
- for (Iterator it = cachedArchives.keySet().iterator(); it.hasNext();) {
|
|
|
|
- String cacheId = (String) it.next();
|
|
|
|
- CacheStatus lcacheStatus = cachedArchives.get(cacheId);
|
|
|
|
- synchronized (lcacheStatus) {
|
|
|
|
- if (lcacheStatus.refcount == 0) {
|
|
|
|
- // delete this cache entry
|
|
|
|
- FileSystem.getLocal(conf).delete(lcacheStatus.localLoadPath, true);
|
|
|
|
- synchronized (baseDirSize) {
|
|
|
|
- Long dirSize = baseDirSize.get(lcacheStatus.baseDir);
|
|
|
|
- if ( dirSize != null ) {
|
|
|
|
- dirSize -= lcacheStatus.size;
|
|
|
|
- baseDirSize.put(lcacheStatus.baseDir, dirSize);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- it.remove();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /*
|
|
|
|
|
|
+ /**
|
|
* Returns the relative path of the dir this cache will be localized in
|
|
* Returns the relative path of the dir this cache will be localized in
|
|
* relative path that this cache will be localized in. For
|
|
* relative path that this cache will be localized in. For
|
|
* hdfs://hostname:port/absolute_path -- the relative path is
|
|
* hdfs://hostname:port/absolute_path -- the relative path is
|
|
* hostname/absolute path -- if it is just /absolute_path -- then the
|
|
* hostname/absolute path -- if it is just /absolute_path -- then the
|
|
* relative path is hostname of DFS this mapred cluster is running
|
|
* relative path is hostname of DFS this mapred cluster is running
|
|
* on/absolute_path
|
|
* on/absolute_path
|
|
|
|
+ * @deprecated Internal to MapReduce framework. Use DistributedCacheManager
|
|
|
|
+ * instead.
|
|
*/
|
|
*/
|
|
public static String makeRelative(URI cache, Configuration conf)
|
|
public static String makeRelative(URI cache, Configuration conf)
|
|
- throws IOException {
|
|
|
|
- String host = cache.getHost();
|
|
|
|
- if (host == null) {
|
|
|
|
- host = cache.getScheme();
|
|
|
|
- }
|
|
|
|
- if (host == null) {
|
|
|
|
- URI defaultUri = FileSystem.get(conf).getUri();
|
|
|
|
- host = defaultUri.getHost();
|
|
|
|
- if (host == null) {
|
|
|
|
- host = defaultUri.getScheme();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- String path = host + cache.getPath();
|
|
|
|
- path = path.replace(":/","/"); // remove windows device colon
|
|
|
|
- return path;
|
|
|
|
- }
|
|
|
|
|
|
+ throws IOException {
|
|
|
|
+ return new TrackerDistributedCacheManager(conf).makeRelative(cache, conf);
|
|
|
|
|
|
- private static Path cacheFilePath(Path p) {
|
|
|
|
- return new Path(p, p.getName());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // the method which actually copies the caches locally and unjars/unzips them
|
|
|
|
- // and does chmod for the files
|
|
|
|
- private static Path localizeCache(Configuration conf,
|
|
|
|
- URI cache, long confFileStamp,
|
|
|
|
- CacheStatus cacheStatus,
|
|
|
|
- FileStatus fileStatus,
|
|
|
|
- boolean isArchive,
|
|
|
|
- Path currentWorkDir,boolean honorSymLinkConf)
|
|
|
|
- throws IOException {
|
|
|
|
- boolean doSymlink = honorSymLinkConf && getSymlink(conf);
|
|
|
|
- if(cache.getFragment() == null) {
|
|
|
|
- doSymlink = false;
|
|
|
|
- }
|
|
|
|
- FileSystem fs = getFileSystem(cache, conf);
|
|
|
|
- String link = currentWorkDir.toString() + Path.SEPARATOR + cache.getFragment();
|
|
|
|
- File flink = new File(link);
|
|
|
|
- if (ifExistsAndFresh(conf, fs, cache, confFileStamp,
|
|
|
|
- cacheStatus, fileStatus)) {
|
|
|
|
- if (isArchive) {
|
|
|
|
- if (doSymlink){
|
|
|
|
- if (!flink.exists())
|
|
|
|
- FileUtil.symLink(cacheStatus.localLoadPath.toString(),
|
|
|
|
- link);
|
|
|
|
- }
|
|
|
|
- return cacheStatus.localLoadPath;
|
|
|
|
- }
|
|
|
|
- else {
|
|
|
|
- if (doSymlink){
|
|
|
|
- if (!flink.exists())
|
|
|
|
- FileUtil.symLink(cacheFilePath(cacheStatus.localLoadPath).toString(),
|
|
|
|
- link);
|
|
|
|
- }
|
|
|
|
- return cacheFilePath(cacheStatus.localLoadPath);
|
|
|
|
- }
|
|
|
|
- } else {
|
|
|
|
- // remove the old archive
|
|
|
|
- // if the old archive cannot be removed since it is being used by another
|
|
|
|
- // job
|
|
|
|
- // return null
|
|
|
|
- if (cacheStatus.refcount > 1 && (cacheStatus.currentStatus == true))
|
|
|
|
- throw new IOException("Cache " + cacheStatus.localLoadPath.toString()
|
|
|
|
- + " is in use and cannot be refreshed");
|
|
|
|
-
|
|
|
|
- FileSystem localFs = FileSystem.getLocal(conf);
|
|
|
|
- localFs.delete(cacheStatus.localLoadPath, true);
|
|
|
|
- synchronized (baseDirSize) {
|
|
|
|
- Long dirSize = baseDirSize.get(cacheStatus.baseDir);
|
|
|
|
- if ( dirSize != null ) {
|
|
|
|
- dirSize -= cacheStatus.size;
|
|
|
|
- baseDirSize.put(cacheStatus.baseDir, dirSize);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- Path parchive = new Path(cacheStatus.localLoadPath,
|
|
|
|
- new Path(cacheStatus.localLoadPath.getName()));
|
|
|
|
-
|
|
|
|
- if (!localFs.mkdirs(cacheStatus.localLoadPath)) {
|
|
|
|
- throw new IOException("Mkdirs failed to create directory " +
|
|
|
|
- cacheStatus.localLoadPath.toString());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- String cacheId = cache.getPath();
|
|
|
|
- fs.copyToLocalFile(new Path(cacheId), parchive);
|
|
|
|
- if (isArchive) {
|
|
|
|
- String tmpArchive = parchive.toString().toLowerCase();
|
|
|
|
- File srcFile = new File(parchive.toString());
|
|
|
|
- File destDir = new File(parchive.getParent().toString());
|
|
|
|
- if (tmpArchive.endsWith(".jar")) {
|
|
|
|
- RunJar.unJar(srcFile, destDir);
|
|
|
|
- } else if (tmpArchive.endsWith(".zip")) {
|
|
|
|
- FileUtil.unZip(srcFile, destDir);
|
|
|
|
- } else if (isTarFile(tmpArchive)) {
|
|
|
|
- FileUtil.unTar(srcFile, destDir);
|
|
|
|
- }
|
|
|
|
- // else will not do anyhting
|
|
|
|
- // and copy the file into the dir as it is
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- long cacheSize = FileUtil.getDU(new File(parchive.getParent().toString()));
|
|
|
|
- cacheStatus.size = cacheSize;
|
|
|
|
- synchronized (baseDirSize) {
|
|
|
|
- Long dirSize = baseDirSize.get(cacheStatus.baseDir);
|
|
|
|
- if( dirSize == null ) {
|
|
|
|
- dirSize = Long.valueOf(cacheSize);
|
|
|
|
- } else {
|
|
|
|
- dirSize += cacheSize;
|
|
|
|
- }
|
|
|
|
- baseDirSize.put(cacheStatus.baseDir, dirSize);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // do chmod here
|
|
|
|
- try {
|
|
|
|
- //Setting recursive permission to grant everyone read and execute
|
|
|
|
- FileUtil.chmod(cacheStatus.baseDir.toString(), "ugo+rx",true);
|
|
|
|
- } catch(InterruptedException e) {
|
|
|
|
- LOG.warn("Exception in chmod" + e.toString());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // update cacheStatus to reflect the newly cached file
|
|
|
|
- cacheStatus.currentStatus = true;
|
|
|
|
- cacheStatus.mtime = getTimestamp(conf, cache);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (isArchive){
|
|
|
|
- if (doSymlink){
|
|
|
|
- if (!flink.exists())
|
|
|
|
- FileUtil.symLink(cacheStatus.localLoadPath.toString(),
|
|
|
|
- link);
|
|
|
|
- }
|
|
|
|
- return cacheStatus.localLoadPath;
|
|
|
|
- }
|
|
|
|
- else {
|
|
|
|
- if (doSymlink){
|
|
|
|
- if (!flink.exists())
|
|
|
|
- FileUtil.symLink(cacheFilePath(cacheStatus.localLoadPath).toString(),
|
|
|
|
- link);
|
|
|
|
- }
|
|
|
|
- return cacheFilePath(cacheStatus.localLoadPath);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static boolean isTarFile(String filename) {
|
|
|
|
- return (filename.endsWith(".tgz") || filename.endsWith(".tar.gz") ||
|
|
|
|
- filename.endsWith(".tar"));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Checks if the cache has already been localized and is fresh
|
|
|
|
- private static boolean ifExistsAndFresh(Configuration conf, FileSystem fs,
|
|
|
|
- URI cache, long confFileStamp,
|
|
|
|
- CacheStatus lcacheStatus,
|
|
|
|
- FileStatus fileStatus)
|
|
|
|
- throws IOException {
|
|
|
|
- // check for existence of the cache
|
|
|
|
- if (lcacheStatus.currentStatus == false) {
|
|
|
|
- return false;
|
|
|
|
- } else {
|
|
|
|
- long dfsFileStamp;
|
|
|
|
- if (fileStatus != null) {
|
|
|
|
- dfsFileStamp = fileStatus.getModificationTime();
|
|
|
|
- } else {
|
|
|
|
- dfsFileStamp = getTimestamp(conf, cache);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // ensure that the file on hdfs hasn't been modified since the job started
|
|
|
|
- if (dfsFileStamp != confFileStamp) {
|
|
|
|
- LOG.fatal("File: " + cache + " has changed on HDFS since job started");
|
|
|
|
- throw new IOException("File: " + cache +
|
|
|
|
- " has changed on HDFS since job started");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (dfsFileStamp != lcacheStatus.mtime) {
|
|
|
|
- // needs refreshing
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return true;
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -516,21 +288,12 @@ public class DistributedCache {
|
|
* @param jobCacheDir the target directory for creating symlinks
|
|
* @param jobCacheDir the target directory for creating symlinks
|
|
* @param workDir the directory in which the symlinks are created
|
|
* @param workDir the directory in which the symlinks are created
|
|
* @throws IOException
|
|
* @throws IOException
|
|
|
|
+ * @deprecated Internal to MapReduce framework. Use DistributedCacheManager
|
|
|
|
+ * instead.
|
|
*/
|
|
*/
|
|
public static void createAllSymlink(Configuration conf, File jobCacheDir, File workDir)
|
|
public static void createAllSymlink(Configuration conf, File jobCacheDir, File workDir)
|
|
throws IOException{
|
|
throws IOException{
|
|
- if ((jobCacheDir == null || !jobCacheDir.isDirectory()) ||
|
|
|
|
- workDir == null || (!workDir.isDirectory())) {
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
- boolean createSymlink = getSymlink(conf);
|
|
|
|
- if (createSymlink){
|
|
|
|
- File[] list = jobCacheDir.listFiles();
|
|
|
|
- for (int i=0; i < list.length; i++){
|
|
|
|
- FileUtil.symLink(list[i].getAbsolutePath(),
|
|
|
|
- new File(workDir, list[i].getName()).toString());
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ TrackerDistributedCacheManager.createAllSymlink(conf, jobCacheDir, workDir);
|
|
}
|
|
}
|
|
|
|
|
|
private static String getFileSysName(URI url) {
|
|
private static String getFileSysName(URI url) {
|
|
@@ -554,7 +317,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Set the configuration with the given set of archives
|
|
|
|
|
|
+ * Set the configuration with the given set of archives. Intended
|
|
|
|
+ * to be used by user code.
|
|
* @param archives The list of archives that need to be localized
|
|
* @param archives The list of archives that need to be localized
|
|
* @param conf Configuration which will be changed
|
|
* @param conf Configuration which will be changed
|
|
*/
|
|
*/
|
|
@@ -564,7 +328,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Set the configuration with the given set of files
|
|
|
|
|
|
+ * Set the configuration with the given set of files. Intended to be
|
|
|
|
+ * used by user code.
|
|
* @param files The list of files that need to be localized
|
|
* @param files The list of files that need to be localized
|
|
* @param conf Configuration which will be changed
|
|
* @param conf Configuration which will be changed
|
|
*/
|
|
*/
|
|
@@ -574,7 +339,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get cache archives set in the Configuration
|
|
|
|
|
|
+ * Get cache archives set in the Configuration. Used by
|
|
|
|
+ * internal DistributedCache and MapReduce code.
|
|
* @param conf The configuration which contains the archives
|
|
* @param conf The configuration which contains the archives
|
|
* @return A URI array of the caches set in the Configuration
|
|
* @return A URI array of the caches set in the Configuration
|
|
* @throws IOException
|
|
* @throws IOException
|
|
@@ -584,18 +350,19 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get cache files set in the Configuration
|
|
|
|
|
|
+ * Get cache files set in the Configuration. Used by internal
|
|
|
|
+ * DistributedCache and MapReduce code.
|
|
* @param conf The configuration which contains the files
|
|
* @param conf The configuration which contains the files
|
|
* @return A URI array of the files set in the Configuration
|
|
* @return A URI array of the files set in the Configuration
|
|
* @throws IOException
|
|
* @throws IOException
|
|
*/
|
|
*/
|
|
-
|
|
|
|
public static URI[] getCacheFiles(Configuration conf) throws IOException {
|
|
public static URI[] getCacheFiles(Configuration conf) throws IOException {
|
|
return StringUtils.stringToURI(conf.getStrings("mapred.cache.files"));
|
|
return StringUtils.stringToURI(conf.getStrings("mapred.cache.files"));
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Return the path array of the localized caches
|
|
|
|
|
|
+ * Return the path array of the localized caches. Intended to be used
|
|
|
|
+ * by user code.
|
|
* @param conf Configuration that contains the localized archives
|
|
* @param conf Configuration that contains the localized archives
|
|
* @return A path array of localized caches
|
|
* @return A path array of localized caches
|
|
* @throws IOException
|
|
* @throws IOException
|
|
@@ -607,7 +374,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Return the path array of the localized files
|
|
|
|
|
|
+ * Return the path array of the localized files. Intended to be used
|
|
|
|
+ * by user code.
|
|
* @param conf Configuration that contains the localized files
|
|
* @param conf Configuration that contains the localized files
|
|
* @return A path array of localized files
|
|
* @return A path array of localized files
|
|
* @throws IOException
|
|
* @throws IOException
|
|
@@ -618,7 +386,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get the timestamps of the archives
|
|
|
|
|
|
+ * Get the timestamps of the archives. Used by internal
|
|
|
|
+ * DistributedCache and MapReduce code.
|
|
* @param conf The configuration which stored the timestamps
|
|
* @param conf The configuration which stored the timestamps
|
|
* @return a string array of timestamps
|
|
* @return a string array of timestamps
|
|
* @throws IOException
|
|
* @throws IOException
|
|
@@ -629,7 +398,8 @@ public class DistributedCache {
|
|
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get the timestamps of the files
|
|
|
|
|
|
+ * Get the timestamps of the files. Used by internal
|
|
|
|
+ * DistributedCache and MapReduce code.
|
|
* @param conf The configuration which stored the timestamps
|
|
* @param conf The configuration which stored the timestamps
|
|
* @return a string array of timestamps
|
|
* @return a string array of timestamps
|
|
* @throws IOException
|
|
* @throws IOException
|
|
@@ -639,7 +409,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * This is to check the timestamp of the archives to be localized
|
|
|
|
|
|
+ * This is to check the timestamp of the archives to be localized.
|
|
|
|
+ * Used by internal MapReduce code.
|
|
* @param conf Configuration which stores the timestamp's
|
|
* @param conf Configuration which stores the timestamp's
|
|
* @param timestamps comma separated list of timestamps of archives.
|
|
* @param timestamps comma separated list of timestamps of archives.
|
|
* The order should be the same as the order in which the archives are added.
|
|
* The order should be the same as the order in which the archives are added.
|
|
@@ -649,7 +420,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * This is to check the timestamp of the files to be localized
|
|
|
|
|
|
+ * This is to check the timestamp of the files to be localized.
|
|
|
|
+ * Used by internal MapReduce code.
|
|
* @param conf Configuration which stores the timestamp's
|
|
* @param conf Configuration which stores the timestamp's
|
|
* @param timestamps comma separated list of timestamps of files.
|
|
* @param timestamps comma separated list of timestamps of files.
|
|
* The order should be the same as the order in which the files are added.
|
|
* The order should be the same as the order in which the files are added.
|
|
@@ -659,7 +431,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Set the conf to contain the location for localized archives
|
|
|
|
|
|
+ * Set the conf to contain the location for localized archives. Used
|
|
|
|
+ * by internal DistributedCache code.
|
|
* @param conf The conf to modify to contain the localized caches
|
|
* @param conf The conf to modify to contain the localized caches
|
|
* @param str a comma separated list of local archives
|
|
* @param str a comma separated list of local archives
|
|
*/
|
|
*/
|
|
@@ -668,7 +441,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Set the conf to contain the location for localized files
|
|
|
|
|
|
+ * Set the conf to contain the location for localized files. Used
|
|
|
|
+ * by internal DistributedCache code.
|
|
* @param conf The conf to modify to contain the localized caches
|
|
* @param conf The conf to modify to contain the localized caches
|
|
* @param str a comma separated list of local files
|
|
* @param str a comma separated list of local files
|
|
*/
|
|
*/
|
|
@@ -677,7 +451,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Add a archives to be localized to the conf
|
|
|
|
|
|
+ * Add a archives to be localized to the conf. Intended to
|
|
|
|
+ * be used by user code.
|
|
* @param uri The uri of the cache to be localized
|
|
* @param uri The uri of the cache to be localized
|
|
* @param conf Configuration to add the cache to
|
|
* @param conf Configuration to add the cache to
|
|
*/
|
|
*/
|
|
@@ -688,7 +463,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Add a file to be localized to the conf
|
|
|
|
|
|
+ * Add a file to be localized to the conf. Intended
|
|
|
|
+ * to be used by user code.
|
|
* @param uri The uri of the cache to be localized
|
|
* @param uri The uri of the cache to be localized
|
|
* @param conf Configuration to add the cache to
|
|
* @param conf Configuration to add the cache to
|
|
*/
|
|
*/
|
|
@@ -700,7 +476,7 @@ public class DistributedCache {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Add an file path to the current set of classpath entries It adds the file
|
|
* Add an file path to the current set of classpath entries It adds the file
|
|
- * to cache as well.
|
|
|
|
|
|
+ * to cache as well. Intended to be used by user code.
|
|
*
|
|
*
|
|
* @param file Path of the file to be added
|
|
* @param file Path of the file to be added
|
|
* @param conf Configuration that contains the classpath setting
|
|
* @param conf Configuration that contains the classpath setting
|
|
@@ -717,7 +493,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get the file entries in classpath as an array of Path
|
|
|
|
|
|
+ * Get the file entries in classpath as an array of Path.
|
|
|
|
+ * Used by internal DistributedCache code.
|
|
*
|
|
*
|
|
* @param conf Configuration that contains the classpath setting
|
|
* @param conf Configuration that contains the classpath setting
|
|
*/
|
|
*/
|
|
@@ -736,7 +513,7 @@ public class DistributedCache {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Add an archive path to the current set of classpath entries. It adds the
|
|
* Add an archive path to the current set of classpath entries. It adds the
|
|
- * archive to cache as well.
|
|
|
|
|
|
+ * archive to cache as well. Intended to be used by user code.
|
|
*
|
|
*
|
|
* @param archive Path of the archive to be added
|
|
* @param archive Path of the archive to be added
|
|
* @param conf Configuration that contains the classpath setting
|
|
* @param conf Configuration that contains the classpath setting
|
|
@@ -754,7 +531,8 @@ public class DistributedCache {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Get the archive entries in classpath as an array of Path
|
|
|
|
|
|
+ * Get the archive entries in classpath as an array of Path.
|
|
|
|
+ * Used by internal DistributedCache code.
|
|
*
|
|
*
|
|
* @param conf Configuration that contains the classpath setting
|
|
* @param conf Configuration that contains the classpath setting
|
|
*/
|
|
*/
|
|
@@ -773,7 +551,8 @@ public class DistributedCache {
|
|
|
|
|
|
/**
|
|
/**
|
|
* This method allows you to create symlinks in the current working directory
|
|
* This method allows you to create symlinks in the current working directory
|
|
- * of the task to all the cache files/archives
|
|
|
|
|
|
+ * of the task to all the cache files/archives.
|
|
|
|
+ * Intended to be used by user code.
|
|
* @param conf the jobconf
|
|
* @param conf the jobconf
|
|
*/
|
|
*/
|
|
public static void createSymlink(Configuration conf){
|
|
public static void createSymlink(Configuration conf){
|
|
@@ -783,6 +562,7 @@ public class DistributedCache {
|
|
/**
|
|
/**
|
|
* This method checks to see if symlinks are to be create for the
|
|
* This method checks to see if symlinks are to be create for the
|
|
* localized cache files in the current working directory
|
|
* localized cache files in the current working directory
|
|
|
|
+ * Used by internal DistributedCache code.
|
|
* @param conf the jobconf
|
|
* @param conf the jobconf
|
|
* @return true if symlinks are to be created- else return false
|
|
* @return true if symlinks are to be created- else return false
|
|
*/
|
|
*/
|
|
@@ -798,7 +578,7 @@ public class DistributedCache {
|
|
* This method checks if there is a conflict in the fragment names
|
|
* This method checks if there is a conflict in the fragment names
|
|
* of the uris. Also makes sure that each uri has a fragment. It
|
|
* of the uris. Also makes sure that each uri has a fragment. It
|
|
* is only to be called if you want to create symlinks for
|
|
* is only to be called if you want to create symlinks for
|
|
- * the various archives and files.
|
|
|
|
|
|
+ * the various archives and files. May be used by user code.
|
|
* @param uriFiles The uri array of urifiles
|
|
* @param uriFiles The uri array of urifiles
|
|
* @param uriArchives the uri array of uri archives
|
|
* @param uriArchives the uri array of uri archives
|
|
*/
|
|
*/
|
|
@@ -840,52 +620,14 @@ public class DistributedCache {
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
|
|
|
|
- private static class CacheStatus {
|
|
|
|
- // false, not loaded yet, true is loaded
|
|
|
|
- boolean currentStatus;
|
|
|
|
-
|
|
|
|
- // the local load path of this cache
|
|
|
|
- Path localLoadPath;
|
|
|
|
-
|
|
|
|
- //the base dir where the cache lies
|
|
|
|
- Path baseDir;
|
|
|
|
-
|
|
|
|
- //the size of this cache
|
|
|
|
- long size;
|
|
|
|
-
|
|
|
|
- // number of instances using this cache
|
|
|
|
- int refcount;
|
|
|
|
-
|
|
|
|
- // the cache-file modification time
|
|
|
|
- long mtime;
|
|
|
|
-
|
|
|
|
- public CacheStatus(Path baseDir, Path localLoadPath) {
|
|
|
|
- super();
|
|
|
|
- this.currentStatus = false;
|
|
|
|
- this.localLoadPath = localLoadPath;
|
|
|
|
- this.refcount = 0;
|
|
|
|
- this.mtime = -1;
|
|
|
|
- this.baseDir = baseDir;
|
|
|
|
- this.size = 0;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Clear the entire contents of the cache and delete the backing files. This
|
|
* Clear the entire contents of the cache and delete the backing files. This
|
|
* should only be used when the server is reinitializing, because the users
|
|
* should only be used when the server is reinitializing, because the users
|
|
* are going to lose their files.
|
|
* are going to lose their files.
|
|
|
|
+ * @deprecated Internal to MapReduce framework. Use DistributedCacheManager
|
|
|
|
+ * instead.
|
|
*/
|
|
*/
|
|
public static void purgeCache(Configuration conf) throws IOException {
|
|
public static void purgeCache(Configuration conf) throws IOException {
|
|
- synchronized (cachedArchives) {
|
|
|
|
- FileSystem localFs = FileSystem.getLocal(conf);
|
|
|
|
- for (Map.Entry<String,CacheStatus> f: cachedArchives.entrySet()) {
|
|
|
|
- try {
|
|
|
|
- localFs.delete(f.getValue().localLoadPath, true);
|
|
|
|
- } catch (IOException ie) {
|
|
|
|
- LOG.debug("Error cleaning up cache", ie);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- cachedArchives.clear();
|
|
|
|
- }
|
|
|
|
|
|
+ new TrackerDistributedCacheManager(conf).purgeCache();
|
|
}
|
|
}
|
|
}
|
|
}
|