|
@@ -24,12 +24,13 @@ import java.net.URISyntaxException;
|
|
|
import java.util.Collection;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.LinkedList;
|
|
|
+import java.util.LinkedHashMap;
|
|
|
import java.util.Map;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
-import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
-import org.apache.hadoop.classification.InterfaceStability;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
|
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
@@ -40,30 +41,100 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
|
|
|
import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
|
|
|
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
+import org.apache.hadoop.yarn.api.records.URL;
|
|
|
+import org.apache.hadoop.yarn.client.api.SharedCacheClient;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
-@InterfaceAudience.Private
|
|
|
-@InterfaceStability.Unstable
|
|
|
+/**
|
|
|
+ * This class is responsible for uploading resources from the client to HDFS
|
|
|
+ * that are associated with a MapReduce job.
|
|
|
+ */
|
|
|
+@Private
|
|
|
+@Unstable
|
|
|
class JobResourceUploader {
|
|
|
protected static final Log LOG = LogFactory.getLog(JobResourceUploader.class);
|
|
|
private final boolean useWildcard;
|
|
|
private final FileSystem jtFs;
|
|
|
+ private SharedCacheClient scClient = null;
|
|
|
+ private SharedCacheConfig scConfig = new SharedCacheConfig();
|
|
|
+ private ApplicationId appId = null;
|
|
|
|
|
|
JobResourceUploader(FileSystem submitFs, boolean useWildcard) {
|
|
|
this.jtFs = submitFs;
|
|
|
this.useWildcard = useWildcard;
|
|
|
}
|
|
|
|
|
|
+ private void initSharedCache(JobID jobid, Configuration conf) {
|
|
|
+ this.scConfig.init(conf);
|
|
|
+ if (this.scConfig.isSharedCacheEnabled()) {
|
|
|
+ this.scClient = createSharedCacheClient(conf);
|
|
|
+ appId = jobIDToAppId(jobid);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /*
|
|
|
+ * We added this method so that we could do the conversion between JobId and
|
|
|
+ * ApplicationId for the shared cache client. This logic is very similar to
|
|
|
+ * the org.apache.hadoop.mapreduce.TypeConverter#toYarn method. We don't use
|
|
|
+ * that because mapreduce-client-core can not depend on
|
|
|
+ * mapreduce-client-common.
|
|
|
+ */
|
|
|
+ private ApplicationId jobIDToAppId(JobID jobId) {
|
|
|
+ return ApplicationId.newInstance(Long.parseLong(jobId.getJtIdentifier()),
|
|
|
+ jobId.getId());
|
|
|
+ }
|
|
|
+
|
|
|
+ private void stopSharedCache() {
|
|
|
+ if (scClient != null) {
|
|
|
+ scClient.stop();
|
|
|
+ scClient = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create, initialize and start a new shared cache client.
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ protected SharedCacheClient createSharedCacheClient(Configuration conf) {
|
|
|
+ SharedCacheClient scc = SharedCacheClient.createSharedCacheClient();
|
|
|
+ scc.init(conf);
|
|
|
+ scc.start();
|
|
|
+ return scc;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Upload and configure files, libjars, jobjars, and archives pertaining to
|
|
|
* the passed job.
|
|
|
- *
|
|
|
+ * <p>
|
|
|
+ * This client will use the shared cache for libjars, files, archives and
|
|
|
+ * jobjars if it is enabled. When shared cache is enabled, it will try to use
|
|
|
+ * the shared cache and fall back to the default behavior when the scm isn't
|
|
|
+ * available.
|
|
|
+ * <p>
|
|
|
+ * 1. For the resources that have been successfully shared, we will continue
|
|
|
+ * to use them in a shared fashion.
|
|
|
+ * <p>
|
|
|
+ * 2. For the resources that weren't in the cache and need to be uploaded by
|
|
|
+ * NM, we won't ask NM to upload them.
|
|
|
+ *
|
|
|
* @param job the job containing the files to be uploaded
|
|
|
* @param submitJobDir the submission directory of the job
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
public void uploadResources(Job job, Path submitJobDir) throws IOException {
|
|
|
+ try {
|
|
|
+ initSharedCache(job.getJobID(), job.getConfiguration());
|
|
|
+ uploadResourcesInternal(job, submitJobDir);
|
|
|
+ } finally {
|
|
|
+ stopSharedCache();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void uploadResourcesInternal(Job job, Path submitJobDir)
|
|
|
+ throws IOException {
|
|
|
Configuration conf = job.getConfiguration();
|
|
|
short replication =
|
|
|
(short) conf.getInt(Job.SUBMIT_REPLICATION,
|
|
@@ -90,6 +161,7 @@ class JobResourceUploader {
|
|
|
+ " already exists!! This is unexpected.Please check what's there in"
|
|
|
+ " that directory");
|
|
|
}
|
|
|
+ // Create the submission directory for the MapReduce job.
|
|
|
submitJobDir = jtFs.makeQualified(submitJobDir);
|
|
|
submitJobDir = new Path(submitJobDir.toUri().getPath());
|
|
|
FsPermission mapredSysPerms =
|
|
@@ -101,20 +173,45 @@ class JobResourceUploader {
|
|
|
disableErasureCodingForPath(jtFs, submitJobDir);
|
|
|
}
|
|
|
|
|
|
+ // Get the resources that have been added via command line arguments in the
|
|
|
+ // GenericOptionsParser (i.e. files, libjars, archives).
|
|
|
Collection<String> files = conf.getStringCollection("tmpfiles");
|
|
|
Collection<String> libjars = conf.getStringCollection("tmpjars");
|
|
|
Collection<String> archives = conf.getStringCollection("tmparchives");
|
|
|
String jobJar = job.getJar();
|
|
|
|
|
|
+ // Merge resources that have been programmatically specified for the shared
|
|
|
+ // cache via the Job API.
|
|
|
+ files.addAll(conf.getStringCollection(MRJobConfig.FILES_FOR_SHARED_CACHE));
|
|
|
+ libjars.addAll(conf.getStringCollection(
|
|
|
+ MRJobConfig.FILES_FOR_CLASSPATH_AND_SHARED_CACHE));
|
|
|
+ archives.addAll(conf
|
|
|
+ .getStringCollection(MRJobConfig.ARCHIVES_FOR_SHARED_CACHE));
|
|
|
+
|
|
|
+
|
|
|
Map<URI, FileStatus> statCache = new HashMap<URI, FileStatus>();
|
|
|
checkLocalizationLimits(conf, files, libjars, archives, jobJar, statCache);
|
|
|
|
|
|
- uploadFiles(conf, files, submitJobDir, mapredSysPerms, replication);
|
|
|
- uploadLibJars(conf, libjars, submitJobDir, mapredSysPerms, replication);
|
|
|
- uploadArchives(conf, archives, submitJobDir, mapredSysPerms, replication);
|
|
|
- uploadJobJar(job, jobJar, submitJobDir, replication);
|
|
|
+ Map<String, Boolean> fileSCUploadPolicies =
|
|
|
+ new LinkedHashMap<String, Boolean>();
|
|
|
+ Map<String, Boolean> archiveSCUploadPolicies =
|
|
|
+ new LinkedHashMap<String, Boolean>();
|
|
|
+
|
|
|
+ uploadFiles(job, files, submitJobDir, mapredSysPerms, replication,
|
|
|
+ fileSCUploadPolicies, statCache);
|
|
|
+ uploadLibJars(job, libjars, submitJobDir, mapredSysPerms, replication,
|
|
|
+ fileSCUploadPolicies, statCache);
|
|
|
+ uploadArchives(job, archives, submitJobDir, mapredSysPerms, replication,
|
|
|
+ archiveSCUploadPolicies, statCache);
|
|
|
+ uploadJobJar(job, jobJar, submitJobDir, replication, statCache);
|
|
|
addLog4jToDistributedCache(job, submitJobDir);
|
|
|
|
|
|
+ // Note, we do not consider resources in the distributed cache for the
|
|
|
+ // shared cache at this time. Only resources specified via the
|
|
|
+ // GenericOptionsParser or the jobjar.
|
|
|
+ Job.setFileSharedCacheUploadPolicies(conf, fileSCUploadPolicies);
|
|
|
+ Job.setArchiveSharedCacheUploadPolicies(conf, archiveSCUploadPolicies);
|
|
|
+
|
|
|
// set the timestamps of the archives and files
|
|
|
// set the public/private visibility of the archives and files
|
|
|
ClientDistributedCacheManager.determineTimestampsAndCacheVisibilities(conf,
|
|
@@ -125,9 +222,11 @@ class JobResourceUploader {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- void uploadFiles(Configuration conf, Collection<String> files,
|
|
|
- Path submitJobDir, FsPermission mapredSysPerms, short submitReplication)
|
|
|
+ void uploadFiles(Job job, Collection<String> files,
|
|
|
+ Path submitJobDir, FsPermission mapredSysPerms, short submitReplication,
|
|
|
+ Map<String, Boolean> fileSCUploadPolicies, Map<URI, FileStatus> statCache)
|
|
|
throws IOException {
|
|
|
+ Configuration conf = job.getConfiguration();
|
|
|
Path filesDir = JobSubmissionFiles.getJobDistCacheFiles(submitJobDir);
|
|
|
if (!files.isEmpty()) {
|
|
|
mkdirs(jtFs, filesDir, mapredSysPerms);
|
|
@@ -140,17 +239,33 @@ class JobResourceUploader {
|
|
|
+ " Argument must be a valid URI: " + tmpFile, e);
|
|
|
}
|
|
|
Path tmp = new Path(tmpURI);
|
|
|
- Path newPath = copyRemoteFiles(filesDir, tmp, conf, submitReplication);
|
|
|
- try {
|
|
|
- URI pathURI = getPathURI(newPath, tmpURI.getFragment());
|
|
|
- DistributedCache.addCacheFile(pathURI, conf);
|
|
|
- } catch (URISyntaxException ue) {
|
|
|
- // should not throw a uri exception
|
|
|
- throw new IOException(
|
|
|
- "Failed to create a URI (URISyntaxException) for the remote path "
|
|
|
- + newPath + ". This was based on the files parameter: "
|
|
|
- + tmpFile,
|
|
|
- ue);
|
|
|
+ URI newURI = null;
|
|
|
+ boolean uploadToSharedCache = false;
|
|
|
+ if (scConfig.isSharedCacheFilesEnabled()) {
|
|
|
+ newURI = useSharedCache(tmpURI, tmp.getName(), statCache, conf, true);
|
|
|
+ if (newURI == null) {
|
|
|
+ uploadToSharedCache = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (newURI == null) {
|
|
|
+ Path newPath =
|
|
|
+ copyRemoteFiles(filesDir, tmp, conf, submitReplication);
|
|
|
+ try {
|
|
|
+ newURI = getPathURI(newPath, tmpURI.getFragment());
|
|
|
+ } catch (URISyntaxException ue) {
|
|
|
+ // should not throw a uri exception
|
|
|
+ throw new IOException(
|
|
|
+ "Failed to create a URI (URISyntaxException) for the"
|
|
|
+ + " remote path " + newPath
|
|
|
+ + ". This was based on the files parameter: " + tmpFile,
|
|
|
+ ue);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ job.addCacheFile(newURI);
|
|
|
+ if (scConfig.isSharedCacheFilesEnabled()) {
|
|
|
+ fileSCUploadPolicies.put(newURI.toString(), uploadToSharedCache);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -159,9 +274,11 @@ class JobResourceUploader {
|
|
|
// Suppress warning for use of DistributedCache (it is everywhere).
|
|
|
@SuppressWarnings("deprecation")
|
|
|
@VisibleForTesting
|
|
|
- void uploadLibJars(Configuration conf, Collection<String> libjars,
|
|
|
- Path submitJobDir, FsPermission mapredSysPerms, short submitReplication)
|
|
|
+ void uploadLibJars(Job job, Collection<String> libjars, Path submitJobDir,
|
|
|
+ FsPermission mapredSysPerms, short submitReplication,
|
|
|
+ Map<String, Boolean> fileSCUploadPolicies, Map<URI, FileStatus> statCache)
|
|
|
throws IOException {
|
|
|
+ Configuration conf = job.getConfiguration();
|
|
|
Path libjarsDir = JobSubmissionFiles.getJobDistCacheLibjars(submitJobDir);
|
|
|
if (!libjars.isEmpty()) {
|
|
|
mkdirs(jtFs, libjarsDir, mapredSysPerms);
|
|
@@ -176,23 +293,53 @@ class JobResourceUploader {
|
|
|
+ " Argument must be a valid URI: " + tmpjars, e);
|
|
|
}
|
|
|
Path tmp = new Path(tmpURI);
|
|
|
- Path newPath =
|
|
|
- copyRemoteFiles(libjarsDir, tmp, conf, submitReplication);
|
|
|
- try {
|
|
|
- URI pathURI = getPathURI(newPath, tmpURI.getFragment());
|
|
|
- if (!foundFragment) {
|
|
|
- foundFragment = pathURI.getFragment() != null;
|
|
|
+ URI newURI = null;
|
|
|
+ boolean uploadToSharedCache = false;
|
|
|
+ boolean fromSharedCache = false;
|
|
|
+ if (scConfig.isSharedCacheLibjarsEnabled()) {
|
|
|
+ newURI = useSharedCache(tmpURI, tmp.getName(), statCache, conf, true);
|
|
|
+ if (newURI == null) {
|
|
|
+ uploadToSharedCache = true;
|
|
|
+ } else {
|
|
|
+ fromSharedCache = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (newURI == null) {
|
|
|
+ Path newPath =
|
|
|
+ copyRemoteFiles(libjarsDir, tmp, conf, submitReplication);
|
|
|
+ try {
|
|
|
+ newURI = getPathURI(newPath, tmpURI.getFragment());
|
|
|
+ } catch (URISyntaxException ue) {
|
|
|
+ // should not throw a uri exception
|
|
|
+ throw new IOException(
|
|
|
+ "Failed to create a URI (URISyntaxException) for the"
|
|
|
+ + " remote path " + newPath
|
|
|
+ + ". This was based on the libjar parameter: " + tmpjars,
|
|
|
+ ue);
|
|
|
}
|
|
|
- DistributedCache.addFileToClassPath(new Path(pathURI.getPath()), conf,
|
|
|
- jtFs, false);
|
|
|
- libjarURIs.add(pathURI);
|
|
|
- } catch (URISyntaxException ue) {
|
|
|
- // should not throw a uri exception
|
|
|
- throw new IOException(
|
|
|
- "Failed to create a URI (URISyntaxException) for the remote path "
|
|
|
- + newPath + ". This was based on the libjar parameter: "
|
|
|
- + tmpjars,
|
|
|
- ue);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!foundFragment) {
|
|
|
+ // We do not count shared cache paths containing fragments as a
|
|
|
+ // "foundFragment." This is because these resources are not in the
|
|
|
+ // staging directory and will be added to the distributed cache
|
|
|
+ // separately.
|
|
|
+ foundFragment = (newURI.getFragment() != null) && !fromSharedCache;
|
|
|
+ }
|
|
|
+ DistributedCache.addFileToClassPath(new Path(newURI.getPath()), conf,
|
|
|
+ jtFs, false);
|
|
|
+ if (fromSharedCache) {
|
|
|
+ // We simply add this URI to the distributed cache. It will not come
|
|
|
+ // from the staging directory (it is in the shared cache), so we
|
|
|
+ // must add it to the cache regardless of the wildcard feature.
|
|
|
+ DistributedCache.addCacheFile(newURI, conf);
|
|
|
+ } else {
|
|
|
+ libjarURIs.add(newURI);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (scConfig.isSharedCacheLibjarsEnabled()) {
|
|
|
+ fileSCUploadPolicies.put(newURI.toString(), uploadToSharedCache);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -210,9 +357,11 @@ class JobResourceUploader {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- void uploadArchives(Configuration conf, Collection<String> archives,
|
|
|
- Path submitJobDir, FsPermission mapredSysPerms, short submitReplication)
|
|
|
- throws IOException {
|
|
|
+ void uploadArchives(Job job, Collection<String> archives,
|
|
|
+ Path submitJobDir, FsPermission mapredSysPerms, short submitReplication,
|
|
|
+ Map<String, Boolean> archiveSCUploadPolicies,
|
|
|
+ Map<URI, FileStatus> statCache) throws IOException {
|
|
|
+ Configuration conf = job.getConfiguration();
|
|
|
Path archivesDir = JobSubmissionFiles.getJobDistCacheArchives(submitJobDir);
|
|
|
if (!archives.isEmpty()) {
|
|
|
mkdirs(jtFs, archivesDir, mapredSysPerms);
|
|
@@ -225,18 +374,34 @@ class JobResourceUploader {
|
|
|
+ " Argument must be a valid URI: " + tmpArchives, e);
|
|
|
}
|
|
|
Path tmp = new Path(tmpURI);
|
|
|
- Path newPath =
|
|
|
- copyRemoteFiles(archivesDir, tmp, conf, submitReplication);
|
|
|
- try {
|
|
|
- URI pathURI = getPathURI(newPath, tmpURI.getFragment());
|
|
|
- DistributedCache.addCacheArchive(pathURI, conf);
|
|
|
- } catch (URISyntaxException ue) {
|
|
|
- // should not throw an uri excpetion
|
|
|
- throw new IOException(
|
|
|
- "Failed to create a URI (URISyntaxException) for the remote path"
|
|
|
- + newPath + ". This was based on the archive parameter: "
|
|
|
- + tmpArchives,
|
|
|
- ue);
|
|
|
+ URI newURI = null;
|
|
|
+ boolean uploadToSharedCache = false;
|
|
|
+ if (scConfig.isSharedCacheArchivesEnabled()) {
|
|
|
+ newURI = useSharedCache(tmpURI, tmp.getName(), statCache, conf, true);
|
|
|
+ if (newURI == null) {
|
|
|
+ uploadToSharedCache = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (newURI == null) {
|
|
|
+ Path newPath =
|
|
|
+ copyRemoteFiles(archivesDir, tmp, conf, submitReplication);
|
|
|
+ try {
|
|
|
+ newURI = getPathURI(newPath, tmpURI.getFragment());
|
|
|
+ } catch (URISyntaxException ue) {
|
|
|
+ // should not throw a uri exception
|
|
|
+ throw new IOException(
|
|
|
+ "Failed to create a URI (URISyntaxException) for the"
|
|
|
+ + " remote path " + newPath
|
|
|
+ + ". This was based on the archive parameter: "
|
|
|
+ + tmpArchives,
|
|
|
+ ue);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ job.addCacheArchive(newURI);
|
|
|
+ if (scConfig.isSharedCacheArchivesEnabled()) {
|
|
|
+ archiveSCUploadPolicies.put(newURI.toString(), uploadToSharedCache);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -244,7 +409,9 @@ class JobResourceUploader {
|
|
|
|
|
|
@VisibleForTesting
|
|
|
void uploadJobJar(Job job, String jobJar, Path submitJobDir,
|
|
|
- short submitReplication) throws IOException {
|
|
|
+ short submitReplication, Map<URI, FileStatus> statCache)
|
|
|
+ throws IOException {
|
|
|
+ Configuration conf = job.getConfiguration();
|
|
|
if (jobJar != null) { // copy jar to JobTracker's fs
|
|
|
// use jar name if job is not named.
|
|
|
if ("".equals(job.getJobName())) {
|
|
@@ -252,12 +419,59 @@ class JobResourceUploader {
|
|
|
}
|
|
|
Path jobJarPath = new Path(jobJar);
|
|
|
URI jobJarURI = jobJarPath.toUri();
|
|
|
- // If the job jar is already in a global fs,
|
|
|
- // we don't need to copy it from local fs
|
|
|
- if (jobJarURI.getScheme() == null || jobJarURI.getScheme().equals("file")) {
|
|
|
- copyJar(jobJarPath, JobSubmissionFiles.getJobJar(submitJobDir),
|
|
|
- submitReplication);
|
|
|
- job.setJar(JobSubmissionFiles.getJobJar(submitJobDir).toString());
|
|
|
+ Path newJarPath = null;
|
|
|
+ boolean uploadToSharedCache = false;
|
|
|
+ if (jobJarURI.getScheme() == null ||
|
|
|
+ jobJarURI.getScheme().equals("file")) {
|
|
|
+ // job jar is on the local file system
|
|
|
+ if (scConfig.isSharedCacheJobjarEnabled()) {
|
|
|
+ // We must have a qualified path for the shared cache client. We can
|
|
|
+ // assume this is for the local filesystem
|
|
|
+ jobJarPath = FileSystem.getLocal(conf).makeQualified(jobJarPath);
|
|
|
+ // Don't add a resource name here because the resource name (i.e.
|
|
|
+ // job.jar directory symlink) will always be hard coded to job.jar for
|
|
|
+ // the job.jar
|
|
|
+ URI newURI =
|
|
|
+ useSharedCache(jobJarPath.toUri(), null, statCache, conf, false);
|
|
|
+ if (newURI == null) {
|
|
|
+ uploadToSharedCache = true;
|
|
|
+ } else {
|
|
|
+ newJarPath = stringToPath(newURI.toString());
|
|
|
+ // The job jar is coming from the shared cache (i.e. a public
|
|
|
+ // place), so we want the job.jar to have a public visibility.
|
|
|
+ conf.setBoolean(MRJobConfig.JOBJAR_VISIBILITY, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (newJarPath == null) {
|
|
|
+ newJarPath = JobSubmissionFiles.getJobJar(submitJobDir);
|
|
|
+ copyJar(jobJarPath, newJarPath, submitReplication);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ // job jar is in a remote file system
|
|
|
+ if (scConfig.isSharedCacheJobjarEnabled()) {
|
|
|
+ // Don't add a resource name here because the resource name (i.e.
|
|
|
+ // job.jar directory symlink) will always be hard coded to job.jar for
|
|
|
+ // the job.jar
|
|
|
+ URI newURI = useSharedCache(jobJarURI, null, statCache, conf, false);
|
|
|
+ if (newURI == null) {
|
|
|
+ uploadToSharedCache = true;
|
|
|
+ newJarPath = jobJarPath;
|
|
|
+ } else {
|
|
|
+ newJarPath = stringToPath(newURI.toString());
|
|
|
+ // The job jar is coming from the shared cache (i.e. a public
|
|
|
+ // place), so we want the job.jar to have a public visibility.
|
|
|
+ conf.setBoolean(MRJobConfig.JOBJAR_VISIBILITY, true);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ // we don't need to upload the jobjar to the staging directory because
|
|
|
+ // it is already in an accessible place
|
|
|
+ newJarPath = jobJarPath;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ job.setJar(newJarPath.toString());
|
|
|
+ if (scConfig.isSharedCacheJobjarEnabled()) {
|
|
|
+ conf.setBoolean(MRJobConfig.JOBJAR_SHARED_CACHE_UPLOAD_POLICY,
|
|
|
+ uploadToSharedCache);
|
|
|
}
|
|
|
} else {
|
|
|
LOG.warn("No job jar file set. User classes may not be found. "
|
|
@@ -267,7 +481,9 @@ class JobResourceUploader {
|
|
|
|
|
|
/**
|
|
|
* Verify that the resources this job is going to localize are within the
|
|
|
- * localization limits.
|
|
|
+ * localization limits. We count all resources towards these limits regardless
|
|
|
+ * of where they are coming from (i.e. local, distributed cache, or shared
|
|
|
+ * cache).
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
void checkLocalizationLimits(Configuration conf, Collection<String> files,
|
|
@@ -464,6 +680,80 @@ class JobResourceUploader {
|
|
|
return newPath;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Checksum a local resource file and call use for that resource with the scm.
|
|
|
+ */
|
|
|
+ private URI useSharedCache(URI sourceFile, String resourceName,
|
|
|
+ Map<URI, FileStatus> statCache, Configuration conf, boolean honorFragment)
|
|
|
+ throws IOException {
|
|
|
+ if (scClient == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ Path filePath = new Path(sourceFile);
|
|
|
+ if (getFileStatus(statCache, conf, filePath).isDirectory()) {
|
|
|
+ LOG.warn("Shared cache does not support directories"
|
|
|
+ + " (see YARN-6097)." + " Will not upload " + filePath
|
|
|
+ + " to the shared cache.");
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ String rn = resourceName;
|
|
|
+ if (honorFragment) {
|
|
|
+ if (sourceFile.getFragment() != null) {
|
|
|
+ rn = sourceFile.getFragment();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // If for whatever reason, we can't even calculate checksum for
|
|
|
+ // a resource, something is really wrong with the file system;
|
|
|
+ // even non-SCM approach won't work. Let us just throw the exception.
|
|
|
+ String checksum = scClient.getFileChecksum(filePath);
|
|
|
+ URL url = null;
|
|
|
+ try {
|
|
|
+ url = scClient.use(this.appId, checksum);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ LOG.warn("Error trying to contact the shared cache manager,"
|
|
|
+ + " disabling the SCMClient for the rest of this job submission", e);
|
|
|
+ /*
|
|
|
+ * If we fail to contact the SCM, we do not use it for the rest of this
|
|
|
+ * JobResourceUploader's life. This prevents us from having to timeout
|
|
|
+ * each time we try to upload a file while the SCM is unavailable. Instead
|
|
|
+ * we timeout/error the first time and quickly revert to the default
|
|
|
+ * behavior without the shared cache. We do this by stopping the shared
|
|
|
+ * cache client and setting it to null.
|
|
|
+ */
|
|
|
+ stopSharedCache();
|
|
|
+ }
|
|
|
+
|
|
|
+ if (url != null) {
|
|
|
+ // Because we deal with URI's in mapreduce, we need to convert the URL to
|
|
|
+ // a URI and add a fragment if necessary.
|
|
|
+ URI uri = null;
|
|
|
+ try {
|
|
|
+ String name = new Path(url.getFile()).getName();
|
|
|
+ if (rn != null && !name.equals(rn)) {
|
|
|
+ // A name was specified that is different then the URL in the shared
|
|
|
+ // cache. Therefore, we need to set the fragment portion of the URI to
|
|
|
+ // preserve the user's desired name. We assume that there is no
|
|
|
+ // existing fragment in the URL since the shared cache manager does
|
|
|
+ // not use fragments.
|
|
|
+ uri = new URI(url.getScheme(), url.getUserInfo(), url.getHost(),
|
|
|
+ url.getPort(), url.getFile(), null, rn);
|
|
|
+ } else {
|
|
|
+ uri = new URI(url.getScheme(), url.getUserInfo(), url.getHost(),
|
|
|
+ url.getPort(), url.getFile(), null, null);
|
|
|
+ }
|
|
|
+ return uri;
|
|
|
+ } catch (URISyntaxException e) {
|
|
|
+ LOG.warn("Error trying to convert URL received from shared cache to"
|
|
|
+ + " a URI: " + url.toString());
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
void copyJar(Path originalJarPath, Path submitJarFile,
|
|
|
short replication) throws IOException {
|