Merge -r 1166954:1166955 from trunk to branch-0.23 to fix MAPREDUCE-2864.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1166957 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arun Murthy 2011-09-09 01:58:40 +00:00
parent d3960b01f7
commit 3dd8d9476b
107 changed files with 1214 additions and 1112 deletions

View File

@ -242,6 +242,9 @@ Release 0.23.0 - Unreleased
MAPREDUCE-2655. Add audit logs to ResourceManager and NodeManager. (Thomas
Graves via acmurthy)
MAPREDUCE-2864. Normalize configuration variable names for YARN. (Robert
Evans via acmurthy)
OPTIMIZATIONS
MAPREDUCE-2026. Make JobTracker.getJobCounters() and

View File

@ -56,12 +56,12 @@ export YARN_CONF_DIR=$HADOOP_CONF_DIR
Step 9) Setup config: for running mapreduce applications, which now are in user land, you need to setup nodemanager with the following configuration in your yarn-site.xml before you start the nodemanager.
<property>
<name>nodemanager.auxiluary.services</name>
<name>yarn.nodemanager.aux-services</name>
<value>mapreduce.shuffle</value>
</property>
<property>
<name>nodemanager.aux.service.mapreduce.shuffle.class</name>
<name>yarn.nodemanager.aux-services.mapreduce.shuffle.class</name>
<value>org.apache.hadoop.mapred.ShuffleHandler</value>
</property>

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;

View File

@ -34,10 +34,10 @@ import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RPC.Server;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.mapred.SortedRanges.Range;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
import org.apache.hadoop.mapreduce.v2.app.TaskHeartbeatHandler;
@ -104,8 +104,8 @@ public class TaskAttemptListenerImpl extends CompositeService
try {
server =
RPC.getServer(TaskUmbilicalProtocol.class, this, "0.0.0.0", 0,
conf.getInt(AMConstants.AM_TASK_LISTENER_THREADS,
AMConstants.DEFAULT_AM_TASK_LISTENER_THREADS),
conf.getInt(MRJobConfig.MR_AM_TASK_LISTENER_THREAD_COUNT,
MRJobConfig.DEFAULT_MR_AM_TASK_LISTENER_THREAD_COUNT),
false, conf, jobTokenSecretManager);
server.start();
InetSocketAddress listenerAddress = server.getListenerAddress();

View File

@ -20,9 +20,6 @@ package org.apache.hadoop.mapred;
// Workaround for ProgressSplitBlock being package access
public class WrappedProgressSplitsBlock extends ProgressSplitsBlock {
public static final int DEFAULT_NUMBER_PROGRESS_SPLITS = 12;
private WrappedPeriodicStatsAccumulator wrappedProgressWallclockTime;
private WrappedPeriodicStatsAccumulator wrappedProgressCPUTime;
private WrappedPeriodicStatsAccumulator wrappedProgressVirtualMemoryKbytes;

View File

@ -37,12 +37,12 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.Counter;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
import org.apache.hadoop.security.UserGroupInformation;
@ -140,7 +140,7 @@ public class JobHistoryEventHandler extends AbstractService
LOG.info("Creating intermediate history logDir: ["
+ doneDirPath
+ "] + based on conf. Should ideally be created by the JobHistoryServer: "
+ JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY);
+ MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR);
mkdir(
doneDirFS,
doneDirPath,
@ -154,7 +154,7 @@ public class JobHistoryEventHandler extends AbstractService
String message = "Not creating intermediate history logDir: ["
+ doneDirPath
+ "] based on conf: "
+ JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY
+ MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR
+ ". Either set to true or pre-create this directory with appropriate permissions";
LOG.error(message);
throw new YarnException(message);

View File

@ -1,74 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import org.apache.hadoop.mapreduce.v2.MRConstants;
public interface AMConstants {
public static final String CONTAINERLAUNCHER_THREADPOOL_SIZE =
"yarn.mapreduce.containerlauncher.threadpool-size";
public static final String AM_RM_SCHEDULE_INTERVAL =
"yarn.appMaster.scheduler.interval";
public static final int DEFAULT_AM_RM_SCHEDULE_INTERVAL = 2000;
public static final String AM_TASK_LISTENER_THREADS =
MRConstants.YARN_MR_PREFIX + "task.listener.threads";
public static final int DEFAULT_AM_TASK_LISTENER_THREADS = 10;
public static final String AM_JOB_CLIENT_THREADS =
MRConstants.YARN_MR_PREFIX + "job.client.threads";
public static final int DEFAULT_AM_JOB_CLIENT_THREADS = 1;
public static final String SPECULATOR_CLASS =
MRConstants.YARN_MR_PREFIX + "speculator.class";
public static final String TASK_RUNTIME_ESTIMATOR_CLASS =
MRConstants.YARN_MR_PREFIX + "task.runtime.estimator.class";
public static final String TASK_ATTEMPT_PROGRESS_RUNTIME_LINEARIZER_CLASS =
MRConstants.YARN_MR_PREFIX + "task.runtime.linearizer.class";
public static final String EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS =
MRConstants.YARN_MR_PREFIX
+ "task.runtime.estimator.exponential.smooth.lambda";
public static final String EXPONENTIAL_SMOOTHING_SMOOTH_RATE =
MRConstants.YARN_MR_PREFIX
+ "task.runtime.estimator.exponential.smooth.smoothsrate";
public static final String RECOVERY_ENABLE = MRConstants.YARN_MR_PREFIX
+ "recovery.enable";
public static final float DEFAULT_REDUCE_RAMP_UP_LIMIT = 0.5f;
public static final String REDUCE_RAMPUP_UP_LIMIT = MRConstants.YARN_MR_PREFIX
+ "reduce.rampup.limit";
public static final float DEFAULT_REDUCE_PREEMPTION_LIMIT = 0.5f;
public static final String REDUCE_PREEMPTION_LIMIT = MRConstants.YARN_MR_PREFIX
+ "reduce.preemption.limit";
public static final String NODE_BLACKLISTING_ENABLE = MRConstants.YARN_MR_PREFIX
+ "node.blacklisting.enable";
}

View File

@ -154,7 +154,7 @@ public class MRAppMaster extends CompositeService {
// for an app later
appName = conf.get(MRJobConfig.JOB_NAME, "<missing app name>");
if (conf.getBoolean(AMConstants.RECOVERY_ENABLE, false)
if (conf.getBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, false)
&& startCount > 1) {
LOG.info("Recovery is enabled. Will try to recover from previous life.");
Recovery recoveryServ = new RecoveryService(appID, clock, startCount);
@ -349,7 +349,7 @@ public class MRAppMaster extends CompositeService {
try {
speculatorClass
// "yarn.mapreduce.job.speculator.class"
= conf.getClass(AMConstants.SPECULATOR_CLASS,
= conf.getClass(MRJobConfig.MR_AM_JOB_SPECULATOR,
DefaultSpeculator.class,
Speculator.class);
Constructor<? extends Speculator> speculatorConstructor
@ -360,19 +360,19 @@ public class MRAppMaster extends CompositeService {
return result;
} catch (InstantiationException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
+ MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
throw new YarnException(ex);
} catch (IllegalAccessException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
+ MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
throw new YarnException(ex);
} catch (InvocationTargetException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
+ MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
throw new YarnException(ex);
} catch (NoSuchMethodException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
+ MRJobConfig.MR_AM_JOB_SPECULATOR, ex);
throw new YarnException(ex);
}
}

View File

@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse;
@ -59,7 +60,6 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
@ -141,8 +141,8 @@ public class MRClientService extends AbstractService
server =
rpc.getServer(MRClientProtocol.class, protocolHandler, address,
conf, secretManager,
conf.getInt(AMConstants.AM_JOB_CLIENT_THREADS,
AMConstants.DEFAULT_AM_JOB_CLIENT_THREADS));
conf.getInt(MRJobConfig.MR_AM_JOB_CLIENT_THREAD_COUNT,
MRJobConfig.DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT));
server.start();
this.bindAddress =
NetUtils.createSocketAddr(hostNameResolved.getHostAddress()

View File

@ -94,7 +94,6 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent;
import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials;
@ -981,8 +980,8 @@ public abstract class TaskAttemptImpl implements
try {
if (progressSplitBlock == null) {
progressSplitBlock = new WrappedProgressSplitsBlock(conf.getInt(
JHConfig.JOBHISTORY_TASKPROGRESS_NUMBER_SPLITS_KEY,
WrappedProgressSplitsBlock.DEFAULT_NUMBER_PROGRESS_SPLITS));
MRJobConfig.MR_AM_NUM_PROGRESS_SPLITS,
MRJobConfig.DEFAULT_MR_AM_NUM_PROGRESS_SPLITS));
}
return progressSplitBlock;
} finally {

View File

@ -33,8 +33,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.ShuffleHandler;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunchedEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
@ -102,7 +102,7 @@ public class ContainerLauncherImpl extends AbstractService implements
public void start() {
launcherPool =
new ThreadPoolExecutor(getConfig().getInt(
AMConstants.CONTAINERLAUNCHER_THREADPOOL_SIZE, 10),
MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT, 10),
Integer.MAX_VALUE, 1, TimeUnit.HOURS,
new LinkedBlockingQueue<Runnable>());
launcherPool.prestartAllCoreThreads(); // Wait for work.

View File

@ -25,12 +25,11 @@ import java.util.ArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
@ -98,8 +97,8 @@ public class RMCommunicator extends AbstractService {
public void init(Configuration conf) {
super.init(conf);
rmPollInterval =
conf.getInt(AMConstants.AM_RM_SCHEDULE_INTERVAL,
AMConstants.DEFAULT_AM_RM_SCHEDULE_INTERVAL);
conf.getInt(MRJobConfig.MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS,
MRJobConfig.DEFAULT_MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS);
}
@Override
@ -226,8 +225,8 @@ public class RMCommunicator extends AbstractService {
final YarnRPC rpc = YarnRPC.create(getConfig());
final Configuration conf = new Configuration(getConfig());
final String serviceAddr = conf.get(
YarnConfiguration.SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS);
YarnConfiguration.RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS);
UserGroupInformation currentUser;
try {

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
@ -137,11 +136,11 @@ public class RMContainerAllocator extends RMContainerRequestor
MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART);
maxReduceRampupLimit = conf.getFloat(
AMConstants.REDUCE_RAMPUP_UP_LIMIT,
AMConstants.DEFAULT_REDUCE_RAMP_UP_LIMIT);
MRJobConfig.MR_AM_JOB_REDUCE_RAMPUP_UP_LIMIT,
MRJobConfig.DEFAULT_MR_AM_JOB_REDUCE_RAMP_UP_LIMIT);
maxReducePreemptionLimit = conf.getFloat(
AMConstants.REDUCE_PREEMPTION_LIMIT,
AMConstants.DEFAULT_REDUCE_PREEMPTION_LIMIT);
MRJobConfig.MR_AM_JOB_REDUCE_PREEMPTION_LIMIT,
MRJobConfig.DEFAULT_MR_AM_JOB_REDUCE_PREEMPTION_LIMIT);
RackResolver.init(conf);
}

View File

@ -31,7 +31,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@ -102,7 +101,7 @@ public abstract class RMContainerRequestor extends RMCommunicator {
public void init(Configuration conf) {
super.init(conf);
nodeBlacklistingEnabled =
conf.getBoolean(AMConstants.NODE_BLACKLISTING_ENABLE, true);
conf.getBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true);
LOG.info("nodeBlacklistingEnabled:" + nodeBlacklistingEnabled);
maxTaskFailuresPerNode =
conf.getInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 3);

View File

@ -34,13 +34,13 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
@ -117,7 +117,7 @@ public class DefaultSpeculator extends AbstractService implements
try {
// "yarn.mapreduce.job.task.runtime.estimator.class"
Class<? extends TaskRuntimeEstimator> estimatorClass
= conf.getClass(AMConstants.TASK_RUNTIME_ESTIMATOR_CLASS,
= conf.getClass(MRJobConfig.MR_AM_TASK_ESTIMATOR,
LegacyTaskRuntimeEstimator.class,
TaskRuntimeEstimator.class);
@ -128,16 +128,16 @@ public class DefaultSpeculator extends AbstractService implements
estimator.contextualize(conf, context);
} catch (InstantiationException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
LOG.error("Can't make a speculation runtime extimator", ex);
throw new YarnException(ex);
} catch (IllegalAccessException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
LOG.error("Can't make a speculation runtime extimator", ex);
throw new YarnException(ex);
} catch (InvocationTargetException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
LOG.error("Can't make a speculation runtime extimator", ex);
throw new YarnException(ex);
} catch (NoSuchMethodException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
LOG.error("Can't make a speculation runtime extimator", ex);
throw new YarnException(ex);
}

View File

@ -23,8 +23,8 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
@ -129,18 +129,15 @@ public class ExponentiallySmoothedTaskRuntimeEstimator extends StartEndTimesBase
return vectorRef.get();
}
private static final long DEFAULT_EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS
= 1000L * 60;
@Override
public void contextualize(Configuration conf, AppContext context) {
super.contextualize(conf, context);
lambda
= conf.getLong(AMConstants.EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS,
DEFAULT_EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS);
= conf.getLong(MRJobConfig.MR_AM_TASK_ESTIMATOR_SMOOTH_LAMBDA_MS,
MRJobConfig.DEFAULT_MR_AM_TASK_ESTIMATOR_SMNOOTH_LAMBDA_MS);
smoothedValue
= conf.getBoolean(AMConstants.EXPONENTIAL_SMOOTHING_SMOOTH_RATE, true)
= conf.getBoolean(MRJobConfig.MR_AM_TASK_EXTIMATOR_EXPONENTIAL_RATE_ENABLE, true)
? SmoothedValue.RATE : SmoothedValue.TIME_PER_UNIT_PROGRESS;
}

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@ -62,7 +61,6 @@ import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleaner;
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
@ -134,8 +132,8 @@ public class MRApp extends MRAppMaster {
public Job submit(Configuration conf) throws Exception {
String user = conf.get(MRJobConfig.USER_NAME, "mapred");
conf.set(MRJobConfig.USER_NAME, user);
conf.set(MRConstants.APPS_STAGING_DIR_KEY, testAbsPath.toString());
conf.setBoolean(JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY, true);
conf.set(MRJobConfig.MR_AM_STAGING_DIR, testAbsPath.toString());
conf.setBoolean(MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR, true);
//TODO: fix the bug where the speculator gets events with
//not-fully-constructed objects. For now, disable speculative exec
LOG.info("****DISABLING SPECULATIVE EXECUTION*****");

View File

@ -131,7 +131,7 @@ public class TestRecovery {
//in rerun the 1st map will be recovered from previous run
app = new MRAppWithHistory(2, 1, false, this.getClass().getName(), false, ++runCount);
conf = new Configuration();
conf.setBoolean(AMConstants.RECOVERY_ENABLE, true);
conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);

View File

@ -25,9 +25,6 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface MRConstants {
public static final String YARN_MR_PREFIX = "yarn.mapreduce.job.";
// This should be the directory where splits file gets localized on the node
// running ApplicationMaster.
public static final String JOB_SUBMIT_DIR = "jobSubmitDir";
@ -45,8 +42,6 @@ public interface MRConstants {
public static final String YARN_MAPREDUCE_APP_JAR_PATH =
"$YARN_HOME/modules/" + HADOOP_MAPREDUCE_CLIENT_APP_JAR_NAME;
public static final String APPS_STAGING_DIR_KEY = "yarn.apps.stagingDir";
// The token file for the application. Should contain tokens for access to
// remote file system and may optionally contain application specific tokens.
// For now, generated by the AppManagers and used by NodeManagers and the

View File

@ -0,0 +1,111 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.jobhistory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Stores Job History configuration keys that can be set by administrators of
* the Job History server.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class JHAdminConfig {
/** The prefix to all Job History configuration properties.*/
public static final String MR_HISTORY_PREFIX = "mapreduce.jobhistory.";
/** host:port address for History Server API.*/
public static final String MR_HISTORY_ADDRESS = MR_HISTORY_PREFIX + "address";
public static final String DEFAULT_MR_HISTORY_ADDRESS = "0.0.0.0:10020";
/** If history cleaning should be enabled or not.*/
public static final String MR_HISTORY_CLEANER_ENABLE =
MR_HISTORY_PREFIX + "cleaner.enable";
/** Run the History Cleaner every X ms.*/
public static final String MR_HISTORY_CLEANER_INTERVAL_MS =
MR_HISTORY_PREFIX + "cleaner.interval-ms";
/** The number of threads to handle client API requests.*/
public static final String MR_HISTORY_CLIENT_THREAD_COUNT =
MR_HISTORY_PREFIX + "client.thread-count";
public static final int DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT = 10;
/**
* Size of the date string cache. Effects the number of directories
* which will be scanned to find a job.
*/
public static final String MR_HISTORY_DATESTRING_CACHE_SIZE =
MR_HISTORY_PREFIX + "datestring.cache.size";
/** Equivalent to 0.20 mapreduce.jobhistory.debug.mode */
public static final String MR_HISTORY_DEBUG_MODE =
MR_HISTORY_PREFIX + "debug-mode";
/** Path where history files should be stored for DONE jobs. **/
public static final String MR_HISTORY_DONE_DIR =
MR_HISTORY_PREFIX + "done-dir";
/**
* Path where history files should be stored after a job finished and before
* they are pulled into the job history server.
**/
public static final String MR_HISTORY_INTERMEDIATE_DONE_DIR =
MR_HISTORY_PREFIX + "intermediate-done-dir";
/** Size of the job list cache.*/
public static final String MR_HISTORY_JOBLIST_CACHE_SIZE =
MR_HISTORY_PREFIX + "joblist.cache.size";
/** The location of the Kerberos keytab file.*/
public static final String MR_HISTORY_KEYTAB = MR_HISTORY_PREFIX + "keytab";
/** Size of the loaded job cache.*/
public static final String MR_HISTORY_LOADED_JOB_CACHE_SIZE =
MR_HISTORY_PREFIX + "loadedjobs.cache.size";
/**
* The maximum age of a job history file before it is deleted from the history
* server.
*/
public static final String MR_HISTORY_MAX_AGE_MS =
MR_HISTORY_PREFIX + "max-age-ms";
/**
* Scan for history files to more from intermediate done dir to done dir
* every X ms.
*/
public static final String MR_HISTORY_MOVE_INTERVAL_MS =
MR_HISTORY_PREFIX + "move.interval-ms";
/** The number of threads used to move files.*/
public static final String MR_HISTORY_MOVE_THREAD_COUNT =
MR_HISTORY_PREFIX + "move.thread-count";
/** The Kerberos principal for the history server.*/
public static final String MR_HISTORY_PRINCIPAL =
MR_HISTORY_PREFIX + "principal";
/**The address the history server webapp is on.*/
public static final String MR_HISTORY_WEBAPP_ADDRESS =
MR_HISTORY_PREFIX + "webapp.address";
public static final String DEFAULT_MR_HISTORY_WEBAPP_ADDRESS =
"0.0.0.0:19888";
}

View File

@ -1,117 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.jobhistory;
public class JHConfig {
public static final String HS_PREFIX = "yarn.server.historyserver.";
/** host:port address to which to bind to **/
public static final String HS_BIND_ADDRESS = HS_PREFIX + "address";
public static final String HS_USER_NAME = HS_PREFIX + "kerberos.principal";
public static final String HS_KEYTAB_FILE = HS_PREFIX + "jeytab.file";
public static final String DEFAULT_HS_BIND_ADDRESS = "0.0.0.0:10020";
/** Done Dir for for AppMaster **/
public static final String HISTORY_INTERMEDIATE_DONE_DIR_KEY =
"yarn.historyfile.intermediateDoneDir";
/** Done Dir for for AppMaster **/
public static final String HISTORY_DONE_DIR_KEY =
"yarn.historyfile.doneDir";
/**
* Boolean. Create the base dirs in the JobHistoryEventHandler
* Set to false for multi-user clusters.
*/
public static final String CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY =
"yarn.history.create.intermediate.base.dir";
/** Done Dir for history server. **/
public static final String HISTORY_SERVER_DONE_DIR_KEY =
HS_PREFIX + "historyfile.doneDir";
/**
* Size of the job list cache.
*/
public static final String HISTORY_SERVER_JOBLIST_CACHE_SIZE_KEY =
HS_PREFIX + "joblist.cache.size";
/**
* Size of the loaded job cache.
*/
public static final String HISTORY_SERVER_LOADED_JOB_CACHE_SIZE_KEY =
HS_PREFIX + "loadedjobs.cache.size";
/**
* Size of the date string cache. Effects the number of directories
* which will be scanned to find a job.
*/
public static final String HISTORY_SERVER_DATESTRING_CACHE_SIZE_KEY =
HS_PREFIX + "datestring.cache.size";
/**
* The time interval in milliseconds for the history server
* to wake up and scan for files to be moved.
*/
public static final String HISTORY_SERVER_MOVE_THREAD_INTERVAL =
HS_PREFIX + "move.thread.interval";
/**
* The number of threads used to move files.
*/
public static final String HISTORY_SERVER_NUM_MOVE_THREADS =
HS_PREFIX + "move.threads.count";
// Equivalent to 0.20 mapreduce.jobhistory.debug.mode
public static final String HISTORY_DEBUG_MODE_KEY = HS_PREFIX + "debug.mode";
public static final String HISTORY_MAXAGE =
"yarn.historyfile.maxage";
//TODO Move some of the HistoryServer specific out into a separate configuration class.
public static final String HS_KEYTAB_KEY = HS_PREFIX + "keytab";
public static final String HS_SERVER_PRINCIPAL_KEY = "yarn.historyserver.principal";
public static final String RUN_HISTORY_CLEANER_KEY =
HS_PREFIX + "cleaner.run";
/**
* Run interval for the History Cleaner thread.
*/
public static final String HISTORY_CLEANER_RUN_INTERVAL =
HS_PREFIX + "cleaner.run.interval";
public static final String HS_WEBAPP_BIND_ADDRESS = HS_PREFIX +
"address.webapp";
public static final String DEFAULT_HS_WEBAPP_BIND_ADDRESS =
"0.0.0.0:19888";
public static final String HS_CLIENT_THREADS =
HS_PREFIX + "client.threads";
public static final int DEFAULT_HS_CLIENT_THREADS = 10;
//From JTConfig. May need to be moved elsewhere.
public static final String JOBHISTORY_TASKPROGRESS_NUMBER_SPLITS_KEY =
"mapreduce.jobtracker.jobhistory.task.numberprogresssplits";
public static int DEFAULT_NUMBER_PROGRESS_SPLITS = 12;
}

View File

@ -38,8 +38,8 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.net.NetUtils;
@ -184,9 +184,9 @@ public class JobHistoryUtils {
public static String getConfiguredHistoryIntermediateDoneDirPrefix(
Configuration conf) {
String doneDirPrefix = conf
.get(JHConfig.HISTORY_INTERMEDIATE_DONE_DIR_KEY);
.get(JHAdminConfig.MR_HISTORY_INTERMEDIATE_DONE_DIR);
if (doneDirPrefix == null) {
doneDirPrefix = conf.get(MRConstants.APPS_STAGING_DIR_KEY)
doneDirPrefix = conf.get(MRJobConfig.MR_AM_STAGING_DIR)
+ "/history/done_intermediate";
}
return doneDirPrefix;
@ -199,9 +199,9 @@ public class JobHistoryUtils {
*/
public static String getConfiguredHistoryServerDoneDirPrefix(
Configuration conf) {
String doneDirPrefix = conf.get(JHConfig.HISTORY_DONE_DIR_KEY);
String doneDirPrefix = conf.get(JHAdminConfig.MR_HISTORY_DONE_DIR);
if (doneDirPrefix == null) {
doneDirPrefix = conf.get(MRConstants.APPS_STAGING_DIR_KEY)
doneDirPrefix = conf.get(MRJobConfig.MR_AM_STAGING_DIR)
+ "/history/done";
}
return doneDirPrefix;
@ -220,7 +220,7 @@ public class JobHistoryUtils {
public static boolean shouldCreateNonUserDirectory(Configuration conf) {
// Returning true by default to allow non secure single node clusters to work
// without any configuration change.
return conf.getBoolean(JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY, true);
return conf.getBoolean(MRJobConfig.MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR, true);
}
/**
@ -478,8 +478,8 @@ public class JobHistoryUtils {
public static String getHistoryUrl(Configuration conf, ApplicationId appId)
throws UnknownHostException {
//construct the history url for job
String hsAddress = conf.get(JHConfig.HS_WEBAPP_BIND_ADDRESS,
JHConfig.DEFAULT_HS_WEBAPP_BIND_ADDRESS);
String hsAddress = conf.get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS,
JHAdminConfig.DEFAULT_MR_HISTORY_WEBAPP_ADDRESS);
InetSocketAddress address = NetUtils.createSocketAddr(hsAddress);
StringBuffer sb = new StringBuffer();
if (address.getAddress().isAnyLocalAddress() ||

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.mapreduce.v2.security.client;
import java.lang.annotation.Annotation;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.token.TokenInfo;
@ -44,7 +44,7 @@ public class ClientHSSecurityInfo extends SecurityInfo {
@Override
public String serverPrincipal() {
return JHConfig.HS_SERVER_PRINCIPAL_KEY;
return JHAdminConfig.MR_HISTORY_PRINCIPAL;
}
@Override

View File

@ -32,6 +32,7 @@ import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@ -221,7 +222,7 @@ public class MRApps extends Apps {
private static final String STAGING_CONSTANT = ".staging";
public static Path getStagingAreaDir(Configuration conf, String user) {
return new Path(
conf.get(MRConstants.APPS_STAGING_DIR_KEY) +
conf.get(MRJobConfig.MR_AM_STAGING_DIR) +
Path.SEPARATOR + user + Path.SEPARATOR + STAGING_CONSTANT);
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.util;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
@ -113,7 +114,7 @@ public class TestMRApps {
@Test public void testGetJobFileWithUser() {
Configuration conf = new Configuration();
conf.set(MRConstants.APPS_STAGING_DIR_KEY, "/my/path/to/staging");
conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/my/path/to/staging");
String jobFile = MRApps.getJobFile(conf, "dummy-user", new JobID("dummy-job", 12345));
assertNotNull("getJobFile results in null.", jobFile);
assertEquals("jobFile with specified user is not as expected.",

View File

@ -301,4 +301,103 @@ public interface MRJobConfig {
"mapreduce.ubertask.child.ulimit"; // or mapreduce.uber.ulimit?
public static final String UBERTASK_ENV =
"mapreduce.ubertask.child.env"; // or mapreduce.uber.env?
public static final String MR_PREFIX = "yarn.app.mapreduce.";
public static final String MR_AM_PREFIX = MR_PREFIX + "am.";
/** The staging directory for map reduce.*/
public static final String MR_AM_STAGING_DIR =
MR_AM_PREFIX+"staging-dir";
/** The amount of memory the MR app master needs.*/
public static final String MR_AM_VMEM_MB =
MR_AM_PREFIX+"resource.mb";
public static final int DEFAULT_MR_AM_VMEM_MB = 2048;
/** Command line arguments passed to the MR app master.*/
public static final String MR_AM_COMMAND_OPTS =
MR_AM_PREFIX+"command-opts";
public static final String DEFAULT_MR_AM_COMMAND_OPTS = "-Xmx1536m";
/** Root Logging level passed to the MR app master.*/
public static final String MR_AM_LOG_OPTS =
MR_AM_PREFIX+"log-opts";
public static final String DEFAULT_MR_AM_LOG_OPTS = "INFO";
/**The number of splits when reporting progress in MR*/
public static final String MR_AM_NUM_PROGRESS_SPLITS =
MR_AM_PREFIX+"num-progress-splits";
public static final int DEFAULT_MR_AM_NUM_PROGRESS_SPLITS = 12;
/** Number of threads user to launch containers in the app master.*/
public static final String MR_AM_CONTAINERLAUNCHER_THREAD_COUNT =
MR_AM_PREFIX+"containerlauncher.thread-count";
/** Number of threads to handle job client RPC requests.*/
public static final String MR_AM_JOB_CLIENT_THREAD_COUNT =
MR_AM_PREFIX + "job.client.thread-count";
public static final int DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT = 1;
/** Enable blacklisting of nodes in the job.*/
public static final String MR_AM_JOB_NODE_BLACKLISTING_ENABLE =
MR_AM_PREFIX + "job.node.blacklisting.enable";
/** Enable job recovery.*/
public static final String MR_AM_JOB_RECOVERY_ENABLE =
MR_AM_PREFIX + "job.recovery.enable";
/**
* Limit on the number of reducers that can be preempted to ensure that at
* least one map task can run if it needs to. Percentage between 0.0 and 1.0
*/
public static final String MR_AM_JOB_REDUCE_PREEMPTION_LIMIT =
MR_AM_PREFIX + "job.reduce.preemption.limit";
public static final float DEFAULT_MR_AM_JOB_REDUCE_PREEMPTION_LIMIT = 0.5f;
/**
* Limit reduces starting until a certain percentage of maps have finished.
* Percentage between 0.0 and 1.0
*/
public static final String MR_AM_JOB_REDUCE_RAMPUP_UP_LIMIT =
MR_AM_PREFIX + "job.reduce.rampup.limit";
public static final float DEFAULT_MR_AM_JOB_REDUCE_RAMP_UP_LIMIT = 0.5f;
/** The class that should be used for speculative execution calculations.*/
public static final String MR_AM_JOB_SPECULATOR =
MR_AM_PREFIX + "job.speculator.class";
/** Class used to estimate task resource needs.*/
public static final String MR_AM_TASK_ESTIMATOR =
MR_AM_PREFIX + "job.task.estimator.class";
/** The lambda value in the smoothing function of the task estimator.*/
public static final String MR_AM_TASK_ESTIMATOR_SMOOTH_LAMBDA_MS =
MR_AM_PREFIX
+ "job.task.estimator.exponential.smooth.lambda-ms";
public static final long DEFAULT_MR_AM_TASK_ESTIMATOR_SMNOOTH_LAMBDA_MS =
1000L * 60;
/** true if the smoothing rate should be exponential.*/
public static final String MR_AM_TASK_EXTIMATOR_EXPONENTIAL_RATE_ENABLE =
MR_AM_PREFIX + "job.task.estimator.exponential.smooth.rate";
/** The number of threads used to handle task RPC calls.*/
public static final String MR_AM_TASK_LISTENER_THREAD_COUNT =
MR_AM_PREFIX + "job.task.listener.thread-count";
public static final int DEFAULT_MR_AM_TASK_LISTENER_THREAD_COUNT = 10;
/** How often the AM should send heartbeats to the RM.*/
public static final String MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS =
MR_AM_PREFIX + "scheduler.heartbeat.interval-ms";
public static final int DEFAULT_MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS = 2000;
/**
* Boolean. Create the base dirs in the JobHistoryEventHandler
* Set to false for multi-user clusters. This is an internal config that
* is set by the MR framework and read by it too.
*/
public static final String MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR =
MR_AM_PREFIX + "create-intermediate-jh-base-dir";
}

View File

@ -63,7 +63,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.hs.webapp.HsWebApp;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityInfo;
@ -107,8 +107,8 @@ public class HistoryClientService extends AbstractService {
YarnConfiguration.YARN_SECURITY_INFO,
ClientHSSecurityInfo.class, SecurityInfo.class);
initializeWebApp(getConfig());
String serviceAddr = conf.get(JHConfig.HS_BIND_ADDRESS,
JHConfig.DEFAULT_HS_BIND_ADDRESS);
String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
InetSocketAddress address = NetUtils.createSocketAddr(serviceAddr);
InetAddress hostNameResolved = null;
try {
@ -120,8 +120,8 @@ public class HistoryClientService extends AbstractService {
server =
rpc.getServer(MRClientProtocol.class, protocolHandler, address,
conf, null,
conf.getInt(JHConfig.HS_CLIENT_THREADS,
JHConfig.DEFAULT_HS_CLIENT_THREADS));
conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT,
JHAdminConfig.DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT));
server.start();
this.bindAddress =
NetUtils.createSocketAddr(hostNameResolved.getHostAddress()
@ -133,8 +133,8 @@ public class HistoryClientService extends AbstractService {
private void initializeWebApp(Configuration conf) {
webApp = new HsWebApp(history);
String bindAddress = conf.get(JHConfig.HS_WEBAPP_BIND_ADDRESS,
JHConfig.DEFAULT_HS_WEBAPP_BIND_ADDRESS);
String bindAddress = conf.get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS,
JHAdminConfig.DEFAULT_MR_HISTORY_WEBAPP_ADDRESS);
WebApps.$for("yarn", this).at(bindAddress).start(webApp);
}

View File

@ -57,7 +57,7 @@ import org.apache.hadoop.mapreduce.jobhistory.JobSummary;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
import org.apache.hadoop.yarn.Clock;
@ -184,7 +184,7 @@ public class JobHistory extends AbstractService implements HistoryContext {
this.appAttemptID = RecordFactoryProvider.getRecordFactory(conf)
.newRecordInstance(ApplicationAttemptId.class);
debugMode = conf.getBoolean(JHConfig.HISTORY_DEBUG_MODE_KEY, false);
debugMode = conf.getBoolean(JHAdminConfig.MR_HISTORY_DEBUG_MODE, false);
serialNumberLowDigits = debugMode ? 1 : 3;
serialNumberFormat = ("%0"
+ (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits) + "d");
@ -216,13 +216,13 @@ public class JobHistory extends AbstractService implements HistoryContext {
jobListCacheSize = conf.getInt(JHConfig.HISTORY_SERVER_JOBLIST_CACHE_SIZE_KEY, DEFAULT_JOBLIST_CACHE_SIZE);
loadedJobCacheSize = conf.getInt(JHConfig.HISTORY_SERVER_LOADED_JOB_CACHE_SIZE_KEY, DEFAULT_LOADEDJOB_CACHE_SIZE);
dateStringCacheSize = conf.getInt(JHConfig.HISTORY_SERVER_DATESTRING_CACHE_SIZE_KEY, DEFAULT_DATESTRING_CACHE_SIZE);
jobListCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_JOBLIST_CACHE_SIZE, DEFAULT_JOBLIST_CACHE_SIZE);
loadedJobCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_LOADED_JOB_CACHE_SIZE, DEFAULT_LOADEDJOB_CACHE_SIZE);
dateStringCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_DATESTRING_CACHE_SIZE, DEFAULT_DATESTRING_CACHE_SIZE);
moveThreadInterval =
conf.getLong(JHConfig.HISTORY_SERVER_MOVE_THREAD_INTERVAL,
conf.getLong(JHAdminConfig.MR_HISTORY_MOVE_INTERVAL_MS,
DEFAULT_MOVE_THREAD_INTERVAL);
numMoveThreads = conf.getInt(JHConfig.HISTORY_SERVER_NUM_MOVE_THREADS, DEFAULT_MOVE_THREAD_COUNT);
numMoveThreads = conf.getInt(JHAdminConfig.MR_HISTORY_MOVE_THREAD_COUNT, DEFAULT_MOVE_THREAD_COUNT);
try {
initExisting();
} catch (IOException e) {
@ -260,12 +260,12 @@ public class JobHistory extends AbstractService implements HistoryContext {
moveIntermediateToDoneThread.start();
//Start historyCleaner
boolean startCleanerService = conf.getBoolean(JHConfig.RUN_HISTORY_CLEANER_KEY, true);
boolean startCleanerService = conf.getBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true);
if (startCleanerService) {
long maxAgeOfHistoryFiles = conf.getLong(JHConfig.HISTORY_MAXAGE,
long maxAgeOfHistoryFiles = conf.getLong(JHAdminConfig.MR_HISTORY_MAX_AGE_MS,
DEFAULT_HISTORY_MAX_AGE);
cleanerScheduledExecutor = new ScheduledThreadPoolExecutor(1);
long runInterval = conf.getLong(JHConfig.HISTORY_CLEANER_RUN_INTERVAL,
long runInterval = conf.getLong(JHAdminConfig.MR_HISTORY_CLEANER_INTERVAL_MS,
DEFAULT_RUN_INTERVAL);
cleanerScheduledExecutor
.scheduleAtFixedRate(new HistoryCleaner(maxAgeOfHistoryFiles),

View File

@ -24,7 +24,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.YarnException;
@ -68,8 +68,8 @@ public class JobHistoryServer extends CompositeService {
}
protected void doSecureLogin(Configuration conf) throws IOException {
SecurityUtil.login(conf, JHConfig.HS_KEYTAB_KEY,
JHConfig.HS_SERVER_PRINCIPAL_KEY);
SecurityUtil.login(conf, JHAdminConfig.MR_HISTORY_KEYTAB,
JHAdminConfig.MR_HISTORY_PRINCIPAL);
}
public static void main(String[] args) {

View File

@ -27,7 +27,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.yarn.YarnException;
@ -72,8 +72,8 @@ public class ClientCache {
private MRClientProtocol instantiateHistoryProxy()
throws IOException {
String serviceAddr = conf.get(JHConfig.HS_BIND_ADDRESS,
JHConfig.DEFAULT_HS_BIND_ADDRESS);
String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
LOG.info("Connecting to HistoryServer at: " + serviceAddr);
Configuration myConf = new Configuration(conf);
//TODO This should ideally be using it's own class (instead of ClientRMSecurityInfo)

View File

@ -26,7 +26,6 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
@ -59,7 +58,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@ -86,8 +84,8 @@ public class ResourceMgrDelegate {
YarnRPC rpc = YarnRPC.create(conf);
InetSocketAddress rmAddress =
NetUtils.createSocketAddr(conf.get(
YarnConfiguration.APPSMANAGER_ADDRESS,
YarnConfiguration.DEFAULT_APPSMANAGER_BIND_ADDRESS));
YarnConfiguration.RM_ADDRESS,
YarnConfiguration.DEFAULT_RM_ADDRESS));
LOG.info("Connecting to ResourceManager at " + rmAddress);
Configuration appsManagerServerConf = new Configuration(this.conf);
appsManagerServerConf.setClass(

View File

@ -59,7 +59,6 @@ import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.mapreduce.v2.ClientConstants;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
@ -93,10 +92,6 @@ public class YARNRunner implements ClientProtocol {
private static final Log LOG = LogFactory.getLog(YARNRunner.class);
public static final String YARN_AM_VMEM_MB =
"yarn.am.mapreduce.resource.mb";
private static final int DEFAULT_YARN_AM_VMEM_MB = 2048;
private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private ResourceMgrDelegate resMgrDelegate;
private ClientCache clientCache;
@ -273,7 +268,8 @@ public class YARNRunner implements ClientProtocol {
ApplicationId applicationId = resMgrDelegate.getApplicationId();
appContext.setApplicationId(applicationId);
Resource capability = recordFactory.newRecordInstance(Resource.class);
capability.setMemory(conf.getInt(YARN_AM_VMEM_MB, DEFAULT_YARN_AM_VMEM_MB));
capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
MRJobConfig.DEFAULT_MR_AM_VMEM_MB));
LOG.info("AppMaster capability = " + capability);
appContext.setMasterCapability(capability);
@ -334,11 +330,11 @@ public class YARNRunner implements ClientProtocol {
Vector<CharSequence> vargs = new Vector<CharSequence>(8);
vargs.add(javaHome + "/bin/java");
vargs.add("-Dhadoop.root.logger="
+ conf.get(ClientConstants.MR_APPMASTER_LOG_OPTS,
ClientConstants.DEFAULT_MR_APPMASTER_LOG_OPTS) + ",console");
+ conf.get(MRJobConfig.MR_AM_LOG_OPTS,
MRJobConfig.DEFAULT_MR_AM_LOG_OPTS) + ",console");
vargs.add(conf.get(ClientConstants.MR_APPMASTER_COMMAND_OPTS,
ClientConstants.DEFAULT_MR_APPMASTER_COMMAND_OPTS));
vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS,
MRJobConfig.DEFAULT_MR_AM_COMMAND_OPTS));
// Add { job jar, MR app jar } to classpath.
Map<String, String> environment = new HashMap<String, String>();

View File

@ -1,31 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2;
public interface ClientConstants {
public static final String MR_APPMASTER_COMMAND_OPTS =
"yarn.appMaster.commandOpts";
public static final String DEFAULT_MR_APPMASTER_COMMAND_OPTS = "-Xmx1536m";
public static final String MR_APPMASTER_LOG_OPTS = "yarn.appMaster.logOpts";
public static final String DEFAULT_MR_APPMASTER_LOG_OPTS = "INFO";
}

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.mapred;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Iterator;
import junit.framework.Assert;
@ -64,8 +63,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.YarnException;
@ -122,8 +120,8 @@ public class TestClientRedirect {
Configuration conf = new YarnConfiguration();
conf.set(MRConfig.FRAMEWORK_NAME, "yarn");
conf.set(YarnConfiguration.APPSMANAGER_ADDRESS, RMADDRESS);
conf.set(JHConfig.HS_BIND_ADDRESS, HSHOSTADDRESS);
conf.set(YarnConfiguration.RM_ADDRESS, RMADDRESS);
conf.set(JHAdminConfig.MR_HISTORY_ADDRESS, HSHOSTADDRESS);
RMService rmService = new RMService("test");
rmService.init(conf);
rmService.start();

View File

@ -29,11 +29,10 @@ import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.MiniYARNCluster;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.service.Service;
@ -62,19 +61,21 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
public void init(Configuration conf) {
conf.set(MRConfig.FRAMEWORK_NAME, "yarn");
conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
conf.set(MRConstants.APPS_STAGING_DIR_KEY, new File(getTestWorkDir(),
conf.set(MRJobConfig.MR_AM_STAGING_DIR, new File(getTestWorkDir(),
"apps_staging_dir/${user.name}/").getAbsolutePath());
conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of
// which shuffle doesn't happen
//configure the shuffle service in NM
conf.setStrings(AuxServices.AUX_SERVICES,
conf.setStrings(YarnConfiguration.NM_AUX_SERVICES,
new String[] { ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID });
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT,
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT,
ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID), ShuffleHandler.class,
Service.class);
// Non-standard shuffle port
conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 8083);
conf.setClass(NMConfig.NM_CONTAINER_EXECUTOR_CLASS,
conf.setClass(YarnConfiguration.NM_CONTAINER_EXECUTOR,
DefaultContainerExecutor.class, ContainerExecutor.class);
// TestMRJobs is for testing non-uberized operation only; see TestUberAM

View File

@ -68,9 +68,6 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.YarnServerConfig;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -335,11 +332,11 @@ public class TestMRJobs {
mrCluster.getConfig().set(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos");
mrCluster.getConfig().set(RMConfig.RM_KEYTAB, "/etc/krb5.keytab");
mrCluster.getConfig().set(NMConfig.NM_KEYTAB, "/etc/krb5.keytab");
mrCluster.getConfig().set(YarnConfiguration.RM_SERVER_PRINCIPAL_KEY,
mrCluster.getConfig().set(YarnConfiguration.RM_KEYTAB, "/etc/krb5.keytab");
mrCluster.getConfig().set(YarnConfiguration.NM_KEYTAB, "/etc/krb5.keytab");
mrCluster.getConfig().set(YarnConfiguration.RM_PRINCIPAL,
"rm/sightbusy-lx@LOCALHOST");
mrCluster.getConfig().set(YarnServerConfig.NM_SERVER_PRINCIPAL_KEY,
mrCluster.getConfig().set(YarnConfiguration.NM_PRINCIPAL,
"nm/sightbusy-lx@LOCALHOST");
UserGroupInformation.setConfiguration(mrCluster.getConfig());

View File

@ -67,7 +67,7 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
import org.apache.hadoop.yarn.service.AbstractService;
@ -291,7 +291,7 @@ public class ShuffleHandler extends AbstractService
private final Configuration conf;
private final IndexCache indexCache;
private final LocalDirAllocator lDirAlloc =
new LocalDirAllocator(NMConfig.NM_LOCAL_DIR);
new LocalDirAllocator(YarnConfiguration.NM_LOCAL_DIRS);
private final int port;
public Shuffle(Configuration conf) {

View File

@ -27,50 +27,331 @@ public class YarnConfiguration extends Configuration {
private static final Splitter ADDR_SPLITTER = Splitter.on(':').trimResults();
private static final Joiner JOINER = Joiner.on("");
public static final String RM_PREFIX = "yarn.server.resourcemanager.";
public static final String SCHEDULER_ADDRESS = RM_PREFIX
+ "scheduler.address";
public static final String AM_EXPIRY_INTERVAL = RM_PREFIX
+ "application.expiry.interval";
public static final String DEFAULT_SCHEDULER_BIND_ADDRESS = "0.0.0.0:8030";
public static final String APPSMANAGER_ADDRESS = RM_PREFIX
+ "appsManager.address";
public static final String YARN_SECURITY_INFO =
"yarn.security.info.class.name";
public static final String DEFAULT_APPSMANAGER_BIND_ADDRESS =
"0.0.0.0:8040";
private static final String YARN_DEFAULT_XML_FILE = "yarn-default.xml";
private static final String YARN_SITE_XML_FILE = "yarn-site.xml";
public static final String APPLICATION_MANAGER_PRINCIPAL =
"yarn.jobmanager.user-name";
public static final String RM_WEBAPP_BIND_ADDRESS = RM_PREFIX
+ "webapp.address";
public static final String DEFAULT_RM_WEBAPP_BIND_ADDRESS = "0.0.0.0:8088";
static {
Configuration.addDefaultResource(YARN_DEFAULT_XML_FILE);
Configuration.addDefaultResource(YARN_SITE_XML_FILE);
}
public static final String RM_SERVER_PRINCIPAL_KEY =
"yarn.resourcemanager.principal";
//Configurations
/** ACL of who can view this application.*/
public static final String APPLICATION_ACL_VIEW_APP =
"application.acl-view-job";
"yarn.app.acl.view-job";
/** ACL of who can modify this application.*/
public static final String APPLICATION_ACL_MODIFY_APP =
"application.acl-modify-job";
"yarn.app.acl.modify-job";
/**
* Security info class This is an internal config set and
* read by YARN itself.
*/
public static final String YARN_SECURITY_INFO =
"yarn.security.info.class";
/** Delay before deleting resource to ease debugging of NM issues */
public static final String DEBUG_NM_DELETE_DELAY_SEC =
YarnConfiguration.NM_PREFIX + "delete.debug-delay-sec";
////////////////////////////////
// IPC Configs
////////////////////////////////
public static final String IPC_PREFIX = "yarn.ipc.";
/** Factory to create client IPC classes.*/
public static final String IPC_CLIENT_FACTORY =
IPC_PREFIX + "client.factory.class";
/** Type of serialization to use.*/
public static final String IPC_SERIALIZER_TYPE =
IPC_PREFIX + "serializer.type";
public static final String DEFAULT_IPC_SERIALIZER_TYPE = "protocolbuffers";
/** Factory to create server IPC classes.*/
public static final String IPC_SERVER_FACTORY =
IPC_PREFIX + "server.factory.class";
/** Factory to create IPC exceptions.*/
public static final String IPC_EXCEPTION_FACTORY =
IPC_PREFIX + "exception.factory.class";
/** Factory to create serializeable records.*/
public static final String IPC_RECORD_FACTORY =
IPC_PREFIX + "record.factory.class";
/** RPC class implementation*/
public static final String IPC_RPC_IMPL =
IPC_PREFIX + "rpc.class";
public static final String DEFAULT_IPC_RPC_IMPL =
"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC";
////////////////////////////////
// Resource Manager Configs
////////////////////////////////
public static final String RM_PREFIX = "yarn.resourcemanager.";
/** The address of the applications manager interface in the RM.*/
public static final String RM_ADDRESS =
RM_PREFIX + "address";
public static final String DEFAULT_RM_ADDRESS =
"0.0.0.0:8040";
/** The number of threads used to handle applications manager requests.*/
public static final String RM_CLIENT_THREAD_COUNT =
RM_PREFIX + "client.thread-count";
public static final int DEFAULT_RM_CLIENT_THREAD_COUNT = 10;
/** The expiry interval for application master reporting.*/
public static final String RM_AM_EXPIRY_INTERVAL_MS =
RM_PREFIX + "am.liveness-monitor.expiry-interval-ms";
public static final int DEFAULT_RM_AM_EXPIRY_INTERVAL_MS = 600000;
/** The Kerberos principal for the resource manager.*/
public static final String RM_PRINCIPAL =
RM_PREFIX + "principal";
/** The address of the scheduler interface.*/
public static final String RM_SCHEDULER_ADDRESS =
RM_PREFIX + "scheduler.address";
public static final String DEFAULT_RM_SCHEDULER_ADDRESS = "0.0.0.0:8030";
/** Number of threads to handle scheduler interface.*/
public static final String RM_SCHEDULER_CLIENT_THREAD_COUNT =
RM_PREFIX + "scheduler.client.thread-count";
public static final int DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT = 10;
/** The address of the RM web application.*/
public static final String RM_WEBAPP_ADDRESS =
RM_PREFIX + "webapp.address";
public static final String DEFAULT_RM_WEBAPP_ADDRESS = "0.0.0.0:8088";
public static final String RM_RESOURCE_TRACKER_ADDRESS =
RM_PREFIX + "resource-tracker.address";
public static final String DEFAULT_RM_RESOURCE_TRACKER_ADDRESS =
"0.0.0.0:8025";
/** Are RM acls enabled.*/
public static final String RM_ACL_ENABLE =
RM_PREFIX + "acl.enable";
public static final boolean DEFAULT_RM_ACL_ENABLE = false;
/** ACL of who can be admin of RM.*/
public static final String RM_ADMIN_ACL =
RM_PREFIX + "admin.acl";
public static final String DEFAULT_RM_ADMIN_ACL = "*";
/** The address of the RM admin interface.*/
public static final String RM_ADMIN_ADDRESS =
RM_PREFIX + "admin.address";
public static final String DEFAULT_RM_ADMIN_ADDRESS = "0.0.0.0:8141";
/**Number of threads used to handle RM admin interface.*/
public static final String RM_ADMIN_CLIENT_THREAD_COUNT =
RM_PREFIX + "admin.client.thread-count";
public static final int DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT = 1;
/** How often should the RM check that the AM is still alive.*/
public static final String RM_AM_LIVENESS_MONITOR_INTERVAL_MS =
RM_PREFIX + "amliveliness-monitor.interval-ms";
public static final int DEFAULT_RM_AM_LIVENESS_MONITOR_INTERVAL_MS = 1000;
/** The maximum number of application master retries.*/
public static final String RM_AM_MAX_RETRIES =
RM_PREFIX + "am.max-retries";
public static final int DEFAULT_RM_AM_MAX_RETRIES = 1;
/** How often to check that containers are still alive. */
public static final String RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS =
RM_PREFIX + "container.liveness-monitor.interval-ms";
public static final int DEFAULT_RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS =
600000;
/** The keytab for the resource manager.*/
public static final String RM_KEYTAB =
RM_PREFIX + "keytab";
/** How long to wait until a node manager is considered dead.*/
public static final String RM_NM_EXPIRY_INTERVAL_MS =
RM_PREFIX + "nm.liveness-monitor.expiry-interval-ms";
public static final int DEFAULT_RM_NM_EXPIRY_INTERVAL_MS = 600000;
/** How often to check that node managers are still alive.*/
public static final String RM_NM_LIVENESS_MONITOR_INTERVAL_MS =
RM_PREFIX + "nm.liveness-monitor.interval-ms";
public static final int DEFAULT_RM_NM_LIVENESS_MONITOR_INTERVAL_MS = 1000;
/** Path to file with nodes to include.*/
public static final String RM_NODES_INCLUDE_FILE_PATH =
RM_PREFIX + "nodes.include-path";
public static final String DEFAULT_RM_NODES_INCLUDE_FILE_PATH = "";
/** Path to file with nodes to exclude.*/
public static final String RM_NODES_EXCLUDE_FILE_PATH =
RM_PREFIX + "nodes.exclude-path";
public static final String DEFAULT_RM_NODES_EXCLUDE_FILE_PATH = "";
/** Number of threads to handle resource tracker calls.*/
public static final String RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT =
RM_PREFIX + "resource-tracker.client.thread-count";
public static final int DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT = 10;
/** The class to use as the resource scheduler.*/
public static final String RM_SCHEDULER =
RM_PREFIX + "scheduler.class";
/** The class to use as the persistent store.*/
public static final String RM_STORE = RM_PREFIX + "store.class";
/** The address of the zookeeper instance to use with ZK store.*/
public static final String RM_ZK_STORE_ADDRESS =
RM_PREFIX + "zookeeper-store.address";
/** The zookeeper session timeout for the zookeeper store.*/
public static final String RM_ZK_STORE_TIMEOUT_MS =
RM_PREFIX + "zookeeper-store.session.timeout-ms";
public static final int DEFAULT_RM_ZK_STORE_TIMEOUT_MS = 60000;
/** The maximum number of completed applications RM keeps. */
public static final String RM_MAX_COMPLETED_APPLICATIONS =
RM_PREFIX + "max-completed-applications";
public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000;
////////////////////////////////
// Node Manager Configs
////////////////////////////////
/** Prefix for all node manager configs.*/
public static final String NM_PREFIX = "yarn.nodemanager.";
/** address of node manager IPC.*/
public static final String NM_ADDRESS = NM_PREFIX + "address";
public static final String DEFAULT_NM_ADDRESS = "0.0.0.0:45454";
/** who will execute(launch) the containers.*/
public static final String NM_CONTAINER_EXECUTOR =
NM_PREFIX + "container-executor.class";
/** Number of threads container manager uses.*/
public static final String NM_CONTAINER_MGR_THREAD_COUNT =
NM_PREFIX + "container-manager.thread-count";
public static final int DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT = 5;
/** Number of threads used in cleanup.*/
public static final String NM_DELETE_THREAD_COUNT =
NM_PREFIX + "delete.thread-count";
public static final int DEFAULT_NM_DELETE_THREAD_COUNT = 4;
// TODO: Should this instead be dictated by RM?
/** Heartbeat interval to RM*/
public static final String NM_TO_RM_HEARTBEAT_INTERVAL_MS =
NM_PREFIX + "heartbeat.interval-ms";
public static final int DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS = 1000;
/** Keytab for NM.*/
public static final String NM_KEYTAB = NM_PREFIX + "keytab";
/**List of directories to store localized files in.*/
public static final String NM_LOCAL_DIRS = NM_PREFIX + "local-dirs";
public static final String DEFAULT_NM_LOCAL_DIRS = "/tmp/nm-local-dir";
/** Address where the localizer IPC is.*/
public static final String NM_LOCALIZER_ADDRESS =
NM_PREFIX + "localizer.address";
public static final String DEFAULT_NM_LOCALIZER_ADDRESS = "0.0.0.0:4344";
/** Interval in between cache cleanups.*/
public static final String NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =
NM_PREFIX + "localizer.cache.cleanup.interval-ms";
public static final long DEFAULT_NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS =
10 * 60 * 1000;
/** Target size of localizer cache in MB, per local directory.*/
public static final String NM_LOCALIZER_CACHE_TARGET_SIZE_MB =
NM_PREFIX + "localizer.cache.target-size-mb";
public static final long DEFAULT_NM_LOCALIZER_CACHE_TARGET_SIZE_MB = 10 * 1024;
/** Number of threads to handle localization requests.*/
public static final String NM_LOCALIZER_CLIENT_THREAD_COUNT =
NM_PREFIX + "localizer.client.thread-count";
public static final int DEFAULT_NM_LOCALIZER_CLIENT_THREAD_COUNT = 5;
/** Number of threads to use for localization fetching.*/
public static final String NM_LOCALIZER_FETCH_THREAD_COUNT =
NM_PREFIX + "localizer.fetch.thread-count";
public static final int DEFAULT_NM_LOCALIZER_FETCH_THREAD_COUNT = 4;
/** Where to store container logs.*/
public static final String NM_LOG_DIRS = NM_PREFIX + "log-dirs";
public static final String DEFAULT_NM_LOG_DIRS = "/tmp/logs";
/** Where to aggregate logs to.*/
public static final String NM_REMOTE_APP_LOG_DIR =
NM_PREFIX + "remote-app-log-dir";
public static final String DEFAULT_NM_REMOTE_APP_LOG_DIR = "/tmp/logs";
/** Amount of memory in GB that can be allocated for containers.*/
public static final String NM_VMEM_GB = NM_PREFIX + "resource.memory-gb";
public static final int DEFAULT_NM_VMEM_GB = 8;
/** NM Webapp address.**/
public static final String NM_WEBAPP_ADDRESS = NM_PREFIX + "webapp.address";
public static final String DEFAULT_NM_WEBAPP_ADDRESS = "0.0.0.0:9999";
/** How often to monitor containers.*/
public final static String NM_CONTAINER_MON_INTERVAL_MS =
NM_PREFIX + "container-monitor.interval-ms";
public final static int DEFAULT_NM_CONTAINER_MON_INTERVAL_MS = 3000;
/** Class that calculates containers current resource utilization.*/
public static final String NM_CONTAINER_MON_RESOURCE_CALCULATOR =
NM_PREFIX + "container-monitor.resource-calculator.class";
/** Amount of physical ram to reserve for other applications, -1 disables.*/
public static final String NM_RESERVED_MEMORY_MB =
NM_PREFIX + "reserved.memory-mb";
/** Frequency of running node health script.*/
public static final String NM_HEALTH_CHECK_INTERVAL_MS =
NM_PREFIX + "health-checker.interval-ms";
public static final long DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS = 10 * 60 * 1000;
/** Script time out period.*/
public static final String NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS =
NM_PREFIX + "health-checker.script.timeout-ms";
public static final long DEFAULT_NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS =
2 * DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS;
/** The health check script to run.*/
public static final String NM_HEALTH_CHECK_SCRIPT_PATH =
NM_PREFIX + "health-checker.script.path";
/** The arguments to pass to the health check script.*/
public static final String NM_HEALTH_CHECK_SCRIPT_OPTS =
NM_PREFIX + "health-checker.script.opts";
/** The path to the Linux container executor.*/
public static final String NM_LINUX_CONTAINER_EXECUTOR_PATH =
NM_PREFIX + "linux-container-executor.path";
/** T-file compression types used to compress aggregated logs.*/
public static final String NM_LOG_AGG_COMPRESSION_TYPE =
NM_PREFIX + "log-aggregation.compression-type";
public static final String DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE = "none";
/** The kerberos principal for the node manager.*/
public static final String NM_PRINCIPAL =
NM_PREFIX + "principal";
public static final String NM_AUX_SERVICES =
NM_PREFIX + "aux-services";
public static final String NM_AUX_SERVICE_FMT =
NM_PREFIX + "aux-services.%s.class";
public YarnConfiguration() {
super();
}
@ -83,13 +364,13 @@ public class YarnConfiguration extends Configuration {
}
public static String getRMWebAppURL(Configuration conf) {
String addr = conf.get(RM_WEBAPP_BIND_ADDRESS,
DEFAULT_RM_WEBAPP_BIND_ADDRESS);
String addr = conf.get(YarnConfiguration.RM_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS);
Iterator<String> it = ADDR_SPLITTER.split(addr).iterator();
it.next(); // ignore the bind host
String port = it.next();
// Use apps manager address to figure out the host for webapp
addr = conf.get(APPSMANAGER_ADDRESS, DEFAULT_APPSMANAGER_BIND_ADDRESS);
addr = conf.get(YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS);
String host = ADDR_SPLITTER.split(addr).iterator().next();
return JOINER.join("http://", host, ":", port, "/");
}

View File

@ -23,16 +23,11 @@ import java.lang.reflect.Method;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factories.impl.pb.RecordFactoryPBImpl;
public class RecordFactoryProvider {
public static final String RPC_SERIALIZER_KEY = "org.apache.yarn.ipc.rpc.serializer.property";
public static final String RPC_SERIALIZER_DEFAULT = "protocolbuffers";
public static final String RECORD_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.record.factory.class";
private static Configuration defaultConf;
static {
@ -48,13 +43,13 @@ public class RecordFactoryProvider {
//Users can specify a particular factory by providing a configuration.
conf = defaultConf;
}
String recordFactoryClassName = conf.get(RECORD_FACTORY_CLASS_KEY);
String recordFactoryClassName = conf.get(YarnConfiguration.IPC_RECORD_FACTORY);
if (recordFactoryClassName == null) {
String serializer = conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT);
if (serializer.equals(RPC_SERIALIZER_DEFAULT)) {
String serializer = conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE);
if (serializer.equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) {
return RecordFactoryPBImpl.get();
} else {
throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + RECORD_FACTORY_CLASS_KEY + "] to specify Record factory");
throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_RECORD_FACTORY + "] to specify Record factory");
}
} else {
return (RecordFactory) getFactoryClassInstance(recordFactoryClassName);

View File

@ -25,6 +25,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RpcClientFactory;
import org.apache.hadoop.yarn.factories.RpcServerFactory;
import org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl;
@ -35,13 +36,7 @@ import org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl;
*/
public class RpcFactoryProvider {
private static final Log LOG = LogFactory.getLog(RpcFactoryProvider.class);
//TODO Move these keys to CommonConfigurationKeys
public static final String RPC_SERIALIZER_KEY = "org.apache.yarn.ipc.rpc.serializer.property";
public static final String RPC_SERIALIZER_DEFAULT = "protocolbuffers";
public static final String RPC_CLIENT_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.client.factory.class";
public static final String RPC_SERVER_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.server.factory.class";
private RpcFactoryProvider() {
}
@ -51,12 +46,12 @@ public class RpcFactoryProvider {
if (conf == null) {
conf = new Configuration();
}
String serverFactoryClassName = conf.get(RPC_SERVER_FACTORY_CLASS_KEY);
String serverFactoryClassName = conf.get(YarnConfiguration.IPC_SERVER_FACTORY);
if (serverFactoryClassName == null) {
if (conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT).equals(RPC_SERIALIZER_DEFAULT)) {
if (conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE).equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) {
return RpcServerFactoryPBImpl.get();
} else {
throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + RPC_CLIENT_FACTORY_CLASS_KEY + "][" + RPC_SERVER_FACTORY_CLASS_KEY + "] to specify factories");
throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_CLIENT_FACTORY + "][" + YarnConfiguration.IPC_SERVER_FACTORY + "] to specify factories");
}
} else {
return (RpcServerFactory) getFactoryClassInstance(serverFactoryClassName);
@ -64,12 +59,12 @@ public class RpcFactoryProvider {
}
public static RpcClientFactory getClientFactory(Configuration conf) {
String clientFactoryClassName = conf.get(RPC_CLIENT_FACTORY_CLASS_KEY);
String clientFactoryClassName = conf.get(YarnConfiguration.IPC_CLIENT_FACTORY);
if (clientFactoryClassName == null) {
if (conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT).equals(RPC_SERIALIZER_DEFAULT)) {
if (conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE).equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) {
return RpcClientFactoryPBImpl.get();
} else {
throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + RPC_CLIENT_FACTORY_CLASS_KEY + "][" + RPC_SERVER_FACTORY_CLASS_KEY + "] to specify factories");
throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_CLIENT_FACTORY + "][" + YarnConfiguration.IPC_SERVER_FACTORY + "] to specify factories");
}
} else {
return(RpcClientFactory) getFactoryClassInstance(clientFactoryClassName);

View File

@ -23,16 +23,12 @@ import java.lang.reflect.Method;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.YarnRemoteExceptionFactory;
import org.apache.hadoop.yarn.factories.impl.pb.YarnRemoteExceptionFactoryPBImpl;
public class YarnRemoteExceptionFactoryProvider {
public static final String RPC_SERIALIZER_KEY = "org.apache.yarn.ipc.rpc.serializer.property";
public static final String RPC_SERIALIZER_DEFAULT = "protocolbuffers";
public static final String EXCEPTION_FACTORY_CLASS_KEY = "org.apache.yarn.ipc.exception.factory.class";
private YarnRemoteExceptionFactoryProvider() {
}
@ -40,13 +36,13 @@ public class YarnRemoteExceptionFactoryProvider {
if (conf == null) {
conf = new Configuration();
}
String recordFactoryClassName = conf.get(EXCEPTION_FACTORY_CLASS_KEY);
String recordFactoryClassName = conf.get(YarnConfiguration.IPC_EXCEPTION_FACTORY);
if (recordFactoryClassName == null) {
String serializer = conf.get(RPC_SERIALIZER_KEY, RPC_SERIALIZER_DEFAULT);
if (serializer.equals(RPC_SERIALIZER_DEFAULT)) {
String serializer = conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE, YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE);
if (serializer.equals(YarnConfiguration.DEFAULT_IPC_SERIALIZER_TYPE)) {
return YarnRemoteExceptionFactoryPBImpl.get();
} else {
throw new YarnException("Unknown serializer: [" + conf.get(RPC_SERIALIZER_KEY) + "]. Use keys: [" + EXCEPTION_FACTORY_CLASS_KEY + "] to specify Exception factory");
throw new YarnException("Unknown serializer: [" + conf.get(YarnConfiguration.IPC_SERIALIZER_TYPE) + "]. Use keys: [" + YarnConfiguration.IPC_EXCEPTION_FACTORY + "] to specify Exception factory");
}
} else {
return (YarnRemoteExceptionFactory) getFactoryClassInstance(recordFactoryClassName);

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
/**
* Abstraction to get the RPC implementation for Yarn.
@ -34,13 +35,6 @@ import org.apache.hadoop.yarn.YarnException;
public abstract class YarnRPC {
private static final Log LOG = LogFactory.getLog(YarnRPC.class);
public static final String RPC_CLASSNAME
= "org.apache.hadoop.yarn.ipc.YarnRPC.classname";
//use the default as Hadoop RPC
public static final String DEFAULT_RPC
= "org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC";
public abstract Object getProxy(Class protocol, InetSocketAddress addr,
Configuration conf);
@ -50,10 +44,10 @@ public abstract class YarnRPC {
int numHandlers);
public static YarnRPC create(Configuration conf) {
LOG.info("Creating YarnRPC for " + conf.get(RPC_CLASSNAME));
String clazzName = conf.get(RPC_CLASSNAME);
LOG.info("Creating YarnRPC for " + conf.get(YarnConfiguration.IPC_RPC_IMPL));
String clazzName = conf.get(YarnConfiguration.IPC_RPC_IMPL);
if (clazzName == null) {
clazzName = DEFAULT_RPC;
clazzName = YarnConfiguration.DEFAULT_IPC_RPC_IMPL;
}
try {
return (YarnRPC) Class.forName(clazzName).newInstance();

View File

@ -44,7 +44,7 @@ public class ClientRMSecurityInfo extends SecurityInfo {
@Override
public String serverPrincipal() {
return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY;
return YarnConfiguration.RM_PRINCIPAL;
}
@Override

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@ -70,7 +71,7 @@ public class TestRPC {
private void test(String rpcClass) throws Exception {
Configuration conf = new Configuration();
conf.set(YarnRPC.RPC_CLASSNAME, rpcClass);
conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
YarnRPC rpc = YarnRPC.create(conf);
String bindAddr = "localhost:0";
InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.yarn;
import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RpcClientFactory;
import org.apache.hadoop.yarn.factories.RpcServerFactory;
import org.apache.hadoop.yarn.factories.impl.pb.RpcClientFactoryPBImpl;
@ -42,7 +43,7 @@ public class TestRpcFactoryProvider {
Assert.assertEquals(RpcClientFactoryPBImpl.class, clientFactory.getClass());
Assert.assertEquals(RpcServerFactoryPBImpl.class, serverFactory.getClass());
conf.set(RpcFactoryProvider.RPC_SERIALIZER_KEY, "writable");
conf.set(YarnConfiguration.IPC_SERIALIZER_TYPE, "writable");
try {
clientFactory = RpcFactoryProvider.getClientFactory(conf);
Assert.fail("Expected an exception - unknown serializer");
@ -55,8 +56,8 @@ public class TestRpcFactoryProvider {
}
conf = new Configuration();
conf.set(RpcFactoryProvider.RPC_CLIENT_FACTORY_CLASS_KEY, "NonExistantClass");
conf.set(RpcFactoryProvider.RPC_SERVER_FACTORY_CLASS_KEY, RpcServerFactoryPBImpl.class.getName());
conf.set(YarnConfiguration.IPC_CLIENT_FACTORY, "NonExistantClass");
conf.set(YarnConfiguration.IPC_SERVER_FACTORY, RpcServerFactoryPBImpl.class.getName());
try {
clientFactory = RpcFactoryProvider.getClientFactory(conf);

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.service.AbstractService;
/**
@ -62,28 +63,9 @@ public class NodeHealthCheckerService extends AbstractService {
/** Pattern used for searching in the output of the node health script */
static private final String ERROR_PATTERN = "ERROR";
/* Configuration keys */
public static final String HEALTH_CHECK_SCRIPT_PROPERTY =
"yarn.server.nodemanager.healthchecker.script.path";
public static final String HEALTH_CHECK_INTERVAL_PROPERTY =
"yarn.server.nodemanager.healthchecker.interval";
public static final String HEALTH_CHECK_FAILURE_INTERVAL_PROPERTY =
"yarn.server.nodemanager.healthchecker.script.timeout";
public static final String HEALTH_CHECK_SCRIPT_ARGUMENTS_PROPERTY =
"yarn.server.nodemanager.healthchecker.script.args";
/* end of configuration keys */
/** Time out error message */
static final String NODE_HEALTH_SCRIPT_TIMED_OUT_MSG = "Node health script timed out";
/** Default frequency of running node health script */
private static final long DEFAULT_HEALTH_CHECK_INTERVAL = 10 * 60 * 1000;
/** Default script time out period */
private static final long DEFAULT_HEALTH_SCRIPT_FAILURE_INTERVAL = 2 * DEFAULT_HEALTH_CHECK_INTERVAL;
private boolean isHealthy;
private String healthReport;
@ -224,13 +206,13 @@ public class NodeHealthCheckerService extends AbstractService {
public void init(Configuration conf) {
this.conf = conf;
this.nodeHealthScript =
conf.get(HEALTH_CHECK_SCRIPT_PROPERTY);
this.intervalTime = conf.getLong(HEALTH_CHECK_INTERVAL_PROPERTY,
DEFAULT_HEALTH_CHECK_INTERVAL);
conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH);
this.intervalTime = conf.getLong(YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS,
YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS);
this.scriptTimeout = conf.getLong(
HEALTH_CHECK_FAILURE_INTERVAL_PROPERTY,
DEFAULT_HEALTH_SCRIPT_FAILURE_INTERVAL);
String[] args = conf.getStrings(HEALTH_CHECK_SCRIPT_ARGUMENTS_PROPERTY,
YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS,
YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS);
String[] args = conf.getStrings(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_OPTS,
new String[] {});
timer = new NodeHealthMonitorExecutor(args);
}
@ -340,7 +322,7 @@ public class NodeHealthCheckerService extends AbstractService {
*/
public static boolean shouldRun(Configuration conf) {
String nodeHealthScript =
conf.get(HEALTH_CHECK_SCRIPT_PROPERTY);
conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH);
if (nodeHealthScript == null || nodeHealthScript.trim().isEmpty()) {
return false;
}

View File

@ -43,12 +43,12 @@ public class RMNMSecurityInfoClass extends SecurityInfo {
@Override
public String serverPrincipal() {
return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY;
return YarnConfiguration.RM_PRINCIPAL;
}
@Override
public String clientPrincipal() {
return YarnServerConfig.NM_SERVER_PRINCIPAL_KEY;
return YarnConfiguration.NM_PRINCIPAL;
}
};
}

View File

@ -1,30 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
public class YarnServerConfig {
public static final String NM_SERVER_PRINCIPAL_KEY =
"yarn.nodemanager.principal";
public static final String RESOURCETRACKER_ADDRESS =
YarnConfiguration.RM_PREFIX + "resourcetracker.address";
public static final String DEFAULT_RESOURCETRACKER_BIND_ADDRESS =
"0.0.0.0:8020";
}

View File

@ -1,212 +1,367 @@
<?xml version="1.0"?>
<configuration>
<!-- IPC Configs -->
<property>
<description>Factory to create client IPC classes.</description>
<name>yarn.ipc.client.factory.class</name>
</property>
<property>
<property>
<description>Type of serialization to use.</description>
<name>yarn.ipc.serializer.type</name>
<value>protocolbuffers</value>
</property>
<property>
<description>Factory to create server IPC classes.</description>
<name>yarn.ipc.server.factory.class</name>
</property>
<property>
<description>Factory to create IPC exceptions.</description>
<name>yarn.ipc.exception.factory.class</name>
</property>
<property>
<description>Factory to create serializeable records.</description>
<name>yarn.ipc.record.factory.class</name>
</property>
<property>
<description>RPC class implementation</description>
<name>yarn.ipc.rpc.class</name>
<value>org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC</value>
</property>
<!-- Resource Manager Configs -->
<property>
<description>The address of the applications manager interface in the RM.</description>
<name>yarn.resourcemanager.address</name>
<value>0.0.0.0:8040</value>
</property>
<property>
<description>The number of threads used to handle applications manager requests.</description>
<name>yarn.resourcemanager.client.thread-count</name>
<value>10</value>
</property>
<property>
<description>The expiry interval for application master reporting.</description>
<name>yarn.resourcemanager.am.liveness-monitor.expiry-interval-ms</name>
<value>600000</value>
</property>
<property>
<description>The Kerberos principal for the resource manager.</description>
<name>yarn.resourcemanager.principal</name>
<value>rm/sightbusy-lx@LOCALHOST</value>
</property>
<property>
<name>yarn.nodemanager.principal</name>
<value>nm/sightbusy-lx@LOCALHOST</value>
</property>
<!-- All resourcemanager related configuration properties -->
<property>
<name>yarn.server.resourcemanager.address</name>
<value>0.0.0.0:8020</value>
</property>
<property>
<name>yarn.server.resourcemanager.resourcetracker.address</name>
<value>0.0.0.0:8025</value>
</property>
<property>
<name>yarn.server.resourcemanager.scheduler.address</name>
<description>The address of the scheduler interface.</description>
<name>yarn.resourcemanager.scheduler.address</name>
<value>0.0.0.0:8030</value>
</property>
<property>
<name>yarn.server.resourcemanager.admin.address</name>
<property>
<description>Number of threads to handle scheduler interface.</description>
<name>yarn.resourcemanager.scheduler.client.thread-count</name>
<value>10</value>
</property>
<property>
<description>The address of the RM web application.</description>
<name>yarn.resourcemanager.webapp.address</name>
<value>0.0.0.0:8088</value>
</property>
<property>
<name>yarn.resourcemanager.resource-tracker.address</name>
<value>0.0.0.0:8025</value>
</property>
<property>
<description>Are RM acls enabled.</description>
<name>yarn.resourcemanager.acl.enable</name>
<value>false</value>
</property>
<property>
<description>ACL of who can be admin of RM.</description>
<name>yarn.resourcemanager.admin.acl</name>
<value>*</value>
</property>
<property>
<description>The address of the RM admin interface.</description>
<name>yarn.resourcemanager.admin.address</name>
<value>0.0.0.0:8141</value>
</property>
<property>
<name>yarn.server.resourcemanager.application.max.retries</name>
<description>Number of threads used to handle RM admin interface.</description>
<name>yarn.resourcemanager.admin.client.thread-count</name>
<value>1</value>
<description>The number of times an application will be retried in case
of AM failure.</description>
</property>
<property>
<name>yarn.server.resourcemanager.keytab</name>
<description>How often should the RM check that the AM is still alive.</description>
<name>yarn.resourcemanager.amliveliness-monitor.interval-ms</name>
<value>1000</value>
</property>
<property>
<description>The maximum number of application master retries.</description>
<name>yarn.resourcemanager.am.max-retries</name>
<value>1</value>
</property>
<property>
<description>How often to check that containers are still alive. </description>
<name>yarn.resourcemanager.container.liveness-monitor.interval-ms</name>
<value>600000</value>
</property>
<property>
<description>The keytab for the resource manager.</description>
<name>yarn.resourcemanager.keytab</name>
<value>/etc/krb5.keytab</value>
</property>
<property>
<name>yarn.server.resourcemanager.expire.applications.completed.max</name>
<value>10000</value>
<description>the maximum number of completed applications the RM
keeps in memory
</description>
<description>How long to wait until a node manager is considered dead.</description>
<name>yarn.resourcemanager.nm.liveness-monitor.expiry-interval-ms</name>
<value>600000</value>
</property>
<!-- All nodemanager related configuration properties -->
<property>
<description>How often to check that node managers are still alive.</description>
<name>yarn.resourcemanager.nm.liveness-monitor.interval-ms</name>
<value>1000</value>
</property>
<property>
<name>yarn.server.nodemanager.local-dir</name>
<description>Path to file with nodes to include.</description>
<name>yarn.resourcemanager.nodes.include-path</name>
<value></value>
</property>
<property>
<description>Path to file with nodes to exclude.</description>
<name>yarn.resourcemanager.nodes.exclude-path</name>
<value></value>
</property>
<property>
<description>Number of threads to handle resource tracker calls.</description>
<name>yarn.resourcemanager.resource-tracker.client.thread-count</name>
<value>10</value>
</property>
<property>
<description>The class to use as the resource scheduler.</description>
<name>yarn.resourcemanager.scheduler.class</name>
</property>
<property>
<description>The class to use as the persistent store.</description>
<name>yarn.resourcemanager.store.class</name>
</property>
<property>
<description>The address of the zookeeper instance to use with ZK store.</description>
<name>yarn.resourcemanager.zookeeper-store.address</name>
</property>
<property>
<description>The zookeeper session timeout for the zookeeper store.</description>
<name>yarn.resourcemanager.zookeeper-store.session.timeout-ms</name>
<value>60000</value>
</property>
<property>
<description>The maximum number of completed applications RM keeps. </description>
<name>yarn.resourcemanager.max-completed-applications</name>
<value>10000</value>
</property>
<!-- Node Manager Configs -->
<property>
<description>address of node manager IPC.</description>
<name>yarn.nodemanager.address</name>
<value>0.0.0.0:45454</value>
</property>
<property>
<description>who will execute(launch) the containers.</description>
<name>yarn.nodemanager.container-executor.class</name>
<value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
<!--<value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>-->
</property>
<property>
<description>Number of threads container manager uses.</description>
<name>yarn.nodemanager.container-manager.thread-count</name>
<value>5</value>
</property>
<property>
<description>Number of threads used in cleanup.</description>
<name>yarn.nodemanager.delete.thread-count</name>
<value>4</value>
</property>
<property>
<description>Heartbeat interval to RM</description>
<name>yarn.nodemanager.heartbeat.interval-ms</name>
<value>1000</value>
</property>
<property>
<description>Keytab for NM.</description>
<name>yarn.nodemanager.keytab</name>
<value>/etc/krb5.keytab</value>
</property>
<property>
<description>List of directories to store localized files in.</description>
<name>yarn.nodemanager.local-dirs</name>
<value>/tmp/nm-local-dir</value>
</property>
<property>
<name>yarn.server.nodemanager.log.dir</name>
<description>Address where the localizer IPC is.</description>
<name>yarn.nodemanager.localizer.address</name>
<value>0.0.0.0:4344</value>
</property>
<property>
<description>Interval in between cache cleanups.</description>
<name>yarn.nodemanager.localizer.cache.cleanup.interval-ms</name>
<value>600000</value>
</property>
<property>
<description>Target size of localizer cache in MB, per local directory.</description>
<name>yarn.nodemanager.localizer.cache.target-size-mb</name>
<value>10240</value>
</property>
<property>
<description>Number of threads to handle localization requests.</description>
<name>yarn.nodemanager.localizer.client.thread-count</name>
<value>5</value>
</property>
<property>
<description>Number of threads to use for localization fetching.</description>
<name>yarn.nodemanager.localizer.fetch.thread-count</name>
<value>4</value>
</property>
<property>
<description>Where to store container logs.</description>
<name>yarn.nodemanager.log-dirs</name>
<value>/tmp/logs</value>
</property>
<property>
<name>yarn.apps.stagingDir</name>
<value>/tmp/hadoop-yarn/${user.name}/staging</value>
</property>
<property>
<name>yarn.apps.history.stagingDir</name>
<value>/tmp/hadoop-yarn/${user.name}/staging</value>
</property>
<property>
<name>yarn.server.nodemanager.keytab</name>
<value>/etc/krb5.keytab</value>
<description>Where to aggregate logs to.</description>
<name>yarn.nodemanager.remote-app-log-dir</name>
<value>/tmp/logs</value>
</property>
<property>
<name>yarn.server.nodemanager.container-executor.class</name>
<value>org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor</value>
<!--<value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>-->
</property>
<property><name>NM_HOSTS</name><value>0.0.0.0:45454</value></property>
<property>
<name>yarn.server.nodemanager.address</name>
<value>0.0.0.0:45454</value>
</property>
<!-- HealthChecker's properties -->
<property>
<name>yarn.server.nodemanager.healthchecker.script.path</name>
<value></value>
<description>Location of the node's health-check script on the local
file-system.
</description>
<description>Amount of memory in GB that can be allocated for containers.</description>
<name>yarn.nodemanager.resource.memory-gb</name>
<value>8</value>
</property>
<property>
<name>yarn.server.nodemanager.healthchecker.interval</name>
<value>600000</value>
<description>Frequency of the health-check run by the NodeManager
</description>
<description>NM Webapp address.</description>
<name>yarn.nodemanager.webapp.address</name>
<value>0.0.0.0:9999</value>
</property>
<property>
<name>yarn.server.nodemanager.healthchecker.script.timeout</name>
<value>1200000</value>
<description>Timeout for the health-check run by the NodeManager
</description>
</property>
<property>
<name>yarn.server.nodemanager.healthchecker.script.args</name>
<value></value>
<description>Arguments to be passed to the health-check script run
by the NodeManager</description>
</property>
<property>
<name>yarn.server.nodemanager.healthchecker.script.path</name>
<value></value>
<description>Location of the node's health-check script on the local
file-system.
</description>
</property>
<property>
<name>yarn.server.nodemanager.healthchecker.interval</name>
<value>600000</value>
<description>Frequency of the health-check run by the NodeManager
</description>
</property>
<property>
<name>yarn.server.nodemanager.healthchecker.script.timeout</name>
<value>1200000</value>
<description>Timeout for the health-check run by the NodeManager
</description>
</property>
<property>
<name>yarn.server.nodemanager.healthchecker.script.args</name>
<value></value>
<description>Arguments to be passed to the health-check script run
by the NodeManager</description>
</property>
<!-- End of HealthChecker's properties -->
<!-- ContainerMonitor related properties -->
<property>
<name>yarn.server.nodemanager.containers-monitor.monitoring-interval</name>
<description>How often to monitor containers.</description>
<name>yarn.nodemanager.container-monitor.interval-ms</name>
<value>3000</value>
</property>
<property>
<name>yarn.server.nodemanager.containers-monitor.resourcecalculatorplugin</name>
<description>Class that calculates containers current resource utilization.</description>
<name>yarn.nodemanager.container-monitor.resource-calculator.class</name>
</property>
<property>
<description>Amount of physical ram to reserve for other applications, -1 disables.</description>
<name>yarn.nodemanager.reserved.memory-mb</name>
<value>-1</value>
</property>
<property>
<description>Frequency of running node health script.</description>
<name>yarn.nodemanager.health-checker.interval-ms</name>
<value>600000</value>
</property>
<property>
<description>Script time out period.</description>
<name>yarn.nodemanager.health-checker.script.timeout-ms</name>
<value>1200000</value>
</property>
<property>
<description>The health check script to run.</description>
<name>yarn.nodemanager.health-checker.script.path</name>
<value></value>
</property>
<property>
<name>yarn.server.nodemanager.reserved-physical-memory.mb</name>
<value>-1</value>
</property>
<!-- End of ContainerMonitor related properties -->
<!-- All MRAppMaster related configuration properties -->
<property>
<name>yarn.server.mapreduce-appmanager.attempt-listener.bindAddress</name>
<value>0.0.0.0</value>
<description>The arguments to pass to the health check script.</description>
<name>yarn.nodemanager.health-checker.script.opts</name>
<value></value>
</property>
<property>
<name>yarn.server.mapreduce-appmanager.client-service.bindAddress</name>
<value>0.0.0.0</value>
<description>The path to the Linux container executor.</description>
<name>yarn.nodemanager.linux-container-executor.path</name>
</property>
<property>
<description>T-file compression types used to compress aggregated logs.</description>
<name>yarn.nodemanager.log-aggregation.compression-type</name>
<value>none</value>
</property>
<property>
<description>The kerberos principal for the node manager.</description>
<name>yarn.nodemanager.principal</name>
<value>nm/sightbusy-lx@LOCALHOST</value>
</property>
<property>
<name>yarn.nodemanager.aux-services</name>
<value></value>
<!-- <value>mapreduce.shuffle</value> -->
</property>
<!--Map Reduce configuration-->
<property>
<name>yarn.nodemanager.aux-services.mapreduce.shuffle.class</name>
<value>org.apache.hadoop.mapred.ShuffleHandler</value>
</property>
<property>
<name>mapreduce.job.jar</name>
<value></value>
<!--<value>~/Workspace/eclipse-workspace/yarn/yarn-mapreduce/yarn-mapreduce-app/target/yarn-mapreduce-app-0.23.0-SNAPSHOT.jar</value>-->
<value/>
</property>
<property>
<name>mapreduce.job.hdfs-servers</name>
<value>${fs.default.name}</value>
</property>
<property>
<name>nodemanager.auxiluary.services</name>
<value></value>
<!-- <value>mapreduce.shuffle</value> -->
<name>mapreduce.job.hdfs-servers</name>
<value>${fs.default.name}</value>
</property>
<!--
<property>
<name>nodemanager.aux.service.mapreduce.shuffle.class</name>
<value>org.apache.hadoop.mapred.ShuffleHandler</value>
</property>
-->
</configuration>

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.junit.After;
@ -66,11 +67,11 @@ public class TestNodeHealthService {
private Configuration getConfForNodeHealthScript() {
Configuration conf = new Configuration();
conf.set(NodeHealthCheckerService.HEALTH_CHECK_SCRIPT_PROPERTY,
conf.set(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH,
nodeHealthscriptFile.getAbsolutePath());
conf.setLong(NodeHealthCheckerService.HEALTH_CHECK_INTERVAL_PROPERTY, 500);
conf.setLong(YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS, 500);
conf.setLong(
NodeHealthCheckerService.HEALTH_CHECK_FAILURE_INTERVAL_PROPERTY, 1000);
YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS, 1000);
return conf;
}

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
@ -98,7 +99,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
String appIdStr =
ConverterUtils.toString(container.getContainerID().getAppId());
String[] sLocalDirs =
getConf().getStrings(NMConfig.NM_LOCAL_DIR, NMConfig.DEFAULT_NM_LOCAL_DIR);
getConf().getStrings(YarnConfiguration.NM_LOCAL_DIRS, YarnConfiguration.DEFAULT_NM_LOCAL_DIRS);
for (String sLocalDir : sLocalDirs) {
Path usersdir = new Path(sLocalDir, ContainerLocalizer.USERCACHE);
Path userdir = new Path(usersdir, userName);
@ -358,7 +359,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
throws IOException {
String[] rootLogDirs =
getConf()
.getStrings(NMConfig.NM_LOG_DIR, NMConfig.DEFAULT_NM_LOG_DIR);
.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS);
boolean appLogDirStatus = false;
FsPermission appLogDirPerms = new FsPermission(LOGDIR_PERM);
@ -386,7 +387,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
throws IOException {
String[] rootLogDirs =
getConf()
.getStrings(NMConfig.NM_LOG_DIR, NMConfig.DEFAULT_NM_LOG_DIR);
.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS);
boolean containerLogDirStatus = false;
FsPermission containerLogDirPerms = new FsPermission(LOGDIR_PERM);

View File

@ -28,21 +28,15 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.service.AbstractService;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
public class DeletionService extends AbstractService {
static final Log LOG = LogFactory.getLog(DeletionService.class);
/** Delay before deleting resource to ease debugging of NM issues */
static final String DEBUG_DELAY_SEC =
NMConfig.NM_PREFIX + "debug.delete.delay";
private int debugDelay;
private final ContainerExecutor exec;
private ScheduledThreadPoolExecutor sched;
@ -79,10 +73,10 @@ public class DeletionService extends AbstractService {
public void init(Configuration conf) {
if (conf != null) {
sched = new ScheduledThreadPoolExecutor(
conf.getInt(NM_MAX_DELETE_THREADS, DEFAULT_MAX_DELETE_THREADS));
debugDelay = conf.getInt(DEBUG_DELAY_SEC, 0);
conf.getInt(YarnConfiguration.NM_DELETE_THREAD_COUNT, YarnConfiguration.DEFAULT_NM_DELETE_THREAD_COUNT));
debugDelay = conf.getInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 0);
} else {
sched = new ScheduledThreadPoolExecutor(DEFAULT_MAX_DELETE_THREADS);
sched = new ScheduledThreadPoolExecutor(YarnConfiguration.DEFAULT_NM_DELETE_THREAD_COUNT);
}
sched.setKeepAliveTime(60L, SECONDS);
super.init(conf);

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerDiagnosticsUpdateEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
@ -44,8 +45,6 @@ public class LinuxContainerExecutor extends ContainerExecutor {
.getLog(LinuxContainerExecutor.class);
private String containerExecutorExe;
protected static final String CONTAINER_EXECUTOR_EXEC_KEY =
NMConfig.NM_PREFIX + "linux-container-executor.path";
@Override
public void setConf(Configuration conf) {
@ -98,7 +97,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
new File(hadoopBin, "container-executor").getAbsolutePath();
return null == conf
? defaultPath
: conf.get(CONTAINER_EXECUTOR_EXEC_KEY, defaultPath);
: conf.get(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, defaultPath);
}
@Override

View File

@ -1,103 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.nodemanager;
/** this class stores all the configuration constant keys
* for the nodemanager. All the configuration key variables
* that are going to be used in the nodemanager should be
* stored here. This allows us to see all the configuration
* parameters at one place.
*/
public class NMConfig {
public static final String NM_PREFIX = "yarn.server.nodemanager.";
public static final String DEFAULT_NM_BIND_ADDRESS = "0.0.0.0:45454";
/** host:port address to which to bind to **/
public static final String NM_BIND_ADDRESS = NM_PREFIX + "address";
public static final String DEFAULT_NM_HTTP_BIND_ADDRESS = "0.0.0.0:9999";
/** host:port address to which webserver has to bind to **/
public static final String NM_HTTP_BIND_ADDRESS = NM_PREFIX + "http-address";
public static final String DEFAULT_NM_LOCALIZER_BIND_ADDRESS = "0.0.0.0:4344";
public static final String NM_LOCALIZER_BIND_ADDRESS =
NM_PREFIX + "localizer.address";
public static final String NM_KEYTAB = NM_PREFIX + "keytab";
public static final String NM_CONTAINER_EXECUTOR_CLASS = NM_PREFIX
+ "container-executor.class";
public static final String NM_LOCAL_DIR = NM_PREFIX + "local-dir";
public static final String DEFAULT_NM_LOCAL_DIR = "/tmp/nm-local-dir";
public static final String NM_LOG_DIR = NM_PREFIX + "log.dir"; // TODO: Rename
public static final String DEFAULT_NM_LOG_DIR = "/tmp/logs";
public static final String REMOTE_USER_LOG_DIR = NM_PREFIX
+ "remote-app-log-dir";
public static final String DEFAULT_REMOTE_APP_LOG_DIR = "/tmp/logs";
public static final int DEFAULT_NM_VMEM_GB = 8;
public static final String NM_VMEM_GB = NM_PREFIX + "resource.memory.gb";
// TODO: Should this instead be dictated by RM?
public static final String HEARTBEAT_INTERVAL = NM_PREFIX
+ "heartbeat-interval";
public static final int DEFAULT_HEARTBEAT_INTERVAL = 1000;
public static final String NM_MAX_DELETE_THREADS = NM_PREFIX +
"max.delete.threads";
public static final int DEFAULT_MAX_DELETE_THREADS = 4;
public static final String NM_MAX_PUBLIC_FETCH_THREADS = NM_PREFIX +
"max.public.fetch.threads";
public static final int DEFAULT_MAX_PUBLIC_FETCH_THREADS = 4;
public static final String NM_LOCALIZATION_THREADS =
NM_PREFIX + "localiation.threads";
public static final int DEFAULT_NM_LOCALIZATION_THREADS = 5;
public static final String NM_CONTAINER_MGR_THREADS =
NM_PREFIX + "container.manager.threads";
public static final int DEFAULT_NM_CONTAINER_MGR_THREADS = 5;
public static final String NM_TARGET_CACHE_MB =
NM_PREFIX + "target.cache.size";
public static final long DEFAULT_NM_TARGET_CACHE_MB = 10 * 1024;
public static final String NM_CACHE_CLEANUP_MS =
NM_PREFIX + "target.cache.cleanup.period.ms";
public static final long DEFAULT_NM_CACHE_CLEANUP_MS = 10 * 60 * 1000;
}

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.yarn.server.nodemanager;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_CONTAINER_EXECUTOR_CLASS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_KEYTAB;
import java.io.IOException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -42,7 +39,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.YarnServerConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@ -82,8 +78,8 @@ public class NodeManager extends CompositeService {
}
protected void doSecureLogin() throws IOException {
SecurityUtil.login(getConfig(), NM_KEYTAB,
YarnServerConfig.NM_SERVER_PRINCIPAL_KEY);
SecurityUtil.login(getConfig(), YarnConfiguration.NM_KEYTAB,
YarnConfiguration.NM_PRINCIPAL);
}
@Override
@ -92,7 +88,7 @@ public class NodeManager extends CompositeService {
Context context = new NMContext();
ContainerExecutor exec = ReflectionUtils.newInstance(
conf.getClass(NM_CONTAINER_EXECUTOR_CLASS,
conf.getClass(YarnConfiguration.NM_CONTAINER_EXECUTOR,
DefaultContainerExecutor.class, ContainerExecutor.class), conf);
DeletionService del = new DeletionService(exec);
addService(del);

View File

@ -47,7 +47,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.RMNMSecurityInfoClass;
import org.apache.hadoop.yarn.server.YarnServerConfig;
import org.apache.hadoop.yarn.server.api.ResourceTracker;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
@ -98,12 +97,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
@Override
public synchronized void init(Configuration conf) {
this.rmAddress =
conf.get(YarnServerConfig.RESOURCETRACKER_ADDRESS,
YarnServerConfig.DEFAULT_RESOURCETRACKER_BIND_ADDRESS);
conf.get(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS);
this.heartBeatInterval =
conf.getLong(NMConfig.HEARTBEAT_INTERVAL,
NMConfig.DEFAULT_HEARTBEAT_INTERVAL);
int memory = conf.getInt(NMConfig.NM_VMEM_GB, NMConfig.DEFAULT_NM_VMEM_GB);
conf.getLong(YarnConfiguration.NM_TO_RM_HEARTBEAT_INTERVAL_MS,
YarnConfiguration.DEFAULT_NM_TO_RM_HEARTBEAT_INTERVAL_MS);
int memory = conf.getInt(YarnConfiguration.NM_VMEM_GB, YarnConfiguration.DEFAULT_NM_VMEM_GB);
this.totalResource = recordFactory.newRecordInstance(Resource.class);
this.totalResource.setMemory(memory * 1024);
metrics.addResource(totalResource);
@ -113,13 +112,13 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
@Override
public void start() {
String cmBindAddressStr =
getConfig().get(NMConfig.NM_BIND_ADDRESS,
NMConfig.DEFAULT_NM_BIND_ADDRESS);
getConfig().get(YarnConfiguration.NM_ADDRESS,
YarnConfiguration.DEFAULT_NM_ADDRESS);
InetSocketAddress cmBindAddress =
NetUtils.createSocketAddr(cmBindAddressStr);
String httpBindAddressStr =
getConfig().get(NMConfig.NM_HTTP_BIND_ADDRESS,
NMConfig.DEFAULT_NM_HTTP_BIND_ADDRESS);
getConfig().get(YarnConfiguration.NM_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS);
InetSocketAddress httpBindAddress =
NetUtils.createSocketAddr(httpBindAddressStr);
try {

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.service.Service;
@ -41,9 +42,6 @@ public class AuxServices extends AbstractService
private static final Log LOG = LogFactory.getLog(AuxServices.class);
public static final String AUX_SERVICES = "nodemanager.auxiluary.services";
public static final String AUX_SERVICE_CLASS_FMT =
"nodemanager.aux.service.%s.class";
public final Map<String,AuxiliaryService> serviceMap;
public final Map<String,ByteBuffer> serviceMeta;
@ -85,11 +83,12 @@ public class AuxServices extends AbstractService
@Override
public void init(Configuration conf) {
Collection<String> auxNames = conf.getStringCollection(AUX_SERVICES);
Collection<String> auxNames = conf.getStringCollection(
YarnConfiguration.NM_AUX_SERVICES);
for (final String sName : auxNames) {
try {
Class<? extends AuxiliaryService> sClass = conf.getClass(
String.format(AUX_SERVICE_CLASS_FMT, sName), null,
String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, sName), null,
AuxiliaryService.class);
if (null == sClass) {
throw new RuntimeException("No class defiend for " + sName);

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.yarn.server.nodemanager.containermanager;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_BIND_ADDRESS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_BIND_ADDRESS;
import static org.apache.hadoop.yarn.service.Service.STATE.STARTED;
import java.io.IOException;
@ -31,7 +29,6 @@ import org.apache.avro.ipc.Server;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.DataInputByteBuffer;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials;
@ -67,7 +64,6 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
@ -190,7 +186,7 @@ public class ContainerManagerImpl extends CompositeService implements
@Override
public void init(Configuration conf) {
cmBindAddressStr = NetUtils.createSocketAddr(
conf.get(NM_BIND_ADDRESS, DEFAULT_NM_BIND_ADDRESS));
conf.get(YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS));
super.init(conf);
}
@ -214,8 +210,8 @@ public class ContainerManagerImpl extends CompositeService implements
server =
rpc.getServer(ContainerManager.class, this, cmBindAddressStr, cmConf,
this.containerTokenSecretManager,
cmConf.getInt(NMConfig.NM_CONTAINER_MGR_THREADS,
NMConfig.DEFAULT_NM_CONTAINER_MGR_THREADS));
cmConf.getInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT,
YarnConfiguration.DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT));
LOG.info("ContainerManager started at " + cmBindAddressStr);
server.start();
super.start();

View File

@ -45,10 +45,10 @@ import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
@ -79,7 +79,7 @@ public class ContainerLaunch implements Callable<Integer> {
this.exec = exec;
this.container = container;
this.dispatcher = dispatcher;
this.logDirsSelector = new LocalDirAllocator(NMConfig.NM_LOG_DIR);
this.logDirsSelector = new LocalDirAllocator(YarnConfiguration.NM_LOG_DIRS);
}
@Override
@ -125,7 +125,7 @@ public class ContainerLaunch implements Callable<Integer> {
FileContext lfs = FileContext.getLocalFSFileContext();
LocalDirAllocator lDirAllocator =
new LocalDirAllocator(NMConfig.NM_LOCAL_DIR); // TODO
new LocalDirAllocator(YarnConfiguration.NM_LOCAL_DIRS); // TODO
Path nmPrivateContainerScriptPath =
lDirAllocator.getLocalPathForWrite(
ResourceLocalizationService.NM_PRIVATE_DIR + Path.SEPARATOR
@ -152,8 +152,8 @@ public class ContainerLaunch implements Callable<Integer> {
try {
// /////////// Write out the container-script in the nmPrivate space.
String[] localDirs =
this.conf.getStrings(NMConfig.NM_LOCAL_DIR,
NMConfig.DEFAULT_NM_LOCAL_DIR);
this.conf.getStrings(YarnConfiguration.NM_LOCAL_DIRS,
YarnConfiguration.DEFAULT_NM_LOCAL_DIRS);
List<Path> appDirs = new ArrayList<Path>(localDirs.length);
for (String localDir : localDirs) {
Path usersdir = new Path(localDir, ContainerLocalizer.USERCACHE);

View File

@ -50,18 +50,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import static org.apache.hadoop.fs.CreateFlag.CREATE;
import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_MAX_PUBLIC_FETCH_THREADS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_CACHE_CLEANUP_MS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOCALIZER_BIND_ADDRESS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOCAL_DIR;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOG_DIR;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_TARGET_CACHE_MB;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_CACHE_CLEANUP_MS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOCALIZER_BIND_ADDRESS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOCAL_DIR;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOG_DIR;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_MAX_PUBLIC_FETCH_THREADS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_TARGET_CACHE_MB;
import java.io.IOException;
import java.net.InetSocketAddress;
@ -91,7 +79,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.api.LocalizationProtocol;
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAction;
@ -154,7 +141,7 @@ public class ResourceLocalizationService extends AbstractService
this.exec = exec;
this.dispatcher = dispatcher;
this.delService = delService;
this.localDirsSelector = new LocalDirAllocator(NMConfig.NM_LOCAL_DIR);
this.localDirsSelector = new LocalDirAllocator(YarnConfiguration.NM_LOCAL_DIRS);
this.publicRsrc = new LocalResourcesTrackerImpl(null, dispatcher);
this.cacheCleanup = new ScheduledThreadPoolExecutor(1);
}
@ -174,7 +161,7 @@ public class ResourceLocalizationService extends AbstractService
// TODO queue deletions here, rather than NM init?
FileContext lfs = getLocalFileContext(conf);
String[] sLocalDirs =
conf.getStrings(NM_LOCAL_DIR, DEFAULT_NM_LOCAL_DIR);
conf.getStrings(YarnConfiguration.NM_LOCAL_DIRS, YarnConfiguration.DEFAULT_NM_LOCAL_DIRS);
localDirs = new ArrayList<Path>(sLocalDirs.length);
logDirs = new ArrayList<Path>(sLocalDirs.length);
@ -193,7 +180,7 @@ public class ResourceLocalizationService extends AbstractService
lfs.mkdir(sysdir, NM_PRIVATE_PERM, true);
sysDirs.add(sysdir);
}
String[] sLogdirs = conf.getStrings(NM_LOG_DIR, DEFAULT_NM_LOG_DIR);
String[] sLogdirs = conf.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS);
for (String sLogdir : sLogdirs) {
Path logdir = new Path(sLogdir);
logDirs.add(logdir);
@ -206,11 +193,11 @@ public class ResourceLocalizationService extends AbstractService
logDirs = Collections.unmodifiableList(logDirs);
sysDirs = Collections.unmodifiableList(sysDirs);
cacheTargetSize =
conf.getLong(NM_TARGET_CACHE_MB, DEFAULT_NM_TARGET_CACHE_MB) << 20;
conf.getLong(YarnConfiguration.NM_LOCALIZER_CACHE_TARGET_SIZE_MB, YarnConfiguration.DEFAULT_NM_LOCALIZER_CACHE_TARGET_SIZE_MB) << 20;
cacheCleanupPeriod =
conf.getLong(NM_CACHE_CLEANUP_MS, DEFAULT_NM_CACHE_CLEANUP_MS);
conf.getLong(YarnConfiguration.NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, YarnConfiguration.DEFAULT_NM_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS);
localizationServerAddress = NetUtils.createSocketAddr(
conf.get(NM_LOCALIZER_BIND_ADDRESS, DEFAULT_NM_LOCALIZER_BIND_ADDRESS));
conf.get(YarnConfiguration.NM_LOCALIZER_ADDRESS, YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS));
localizerTracker = new LocalizerTracker(conf);
dispatcher.register(LocalizerEventType.class, localizerTracker);
cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
@ -244,8 +231,8 @@ public class ResourceLocalizationService extends AbstractService
return rpc.getServer(LocalizationProtocol.class, this,
localizationServerAddress, conf, secretManager,
conf.getInt(NMConfig.NM_LOCALIZATION_THREADS,
NMConfig.DEFAULT_NM_LOCALIZATION_THREADS));
conf.getInt(YarnConfiguration.NM_LOCALIZER_CLIENT_THREAD_COUNT,
YarnConfiguration.DEFAULT_NM_LOCALIZER_CLIENT_THREAD_COUNT));
}
@ -496,7 +483,7 @@ public class ResourceLocalizationService extends AbstractService
PublicLocalizer(Configuration conf) {
this(conf, getLocalFileContext(conf),
Executors.newFixedThreadPool(conf.getInt(
NM_MAX_PUBLIC_FETCH_THREADS, DEFAULT_MAX_PUBLIC_FETCH_THREADS)),
YarnConfiguration.NM_LOCALIZER_FETCH_THREAD_COUNT, YarnConfiguration.DEFAULT_NM_LOCALIZER_FETCH_THREAD_COUNT)),
new HashMap<Future<Path>,LocalizerResourceRequestEvent>(),
new HashMap<LocalResourceRequest,List<LocalizerResourceRequestEvent>>());
}

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.file.tfile.TFile;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.util.ConverterUtils;
public class AggregatedLogFormat {
@ -148,8 +149,8 @@ public class AggregatedLogFormat {
// 256KB minBlockSize : Expected log size for each container too
this.writer =
new TFile.Writer(this.fsDataOStream, 256 * 1024, conf.get(
LogAggregationService.LOG_COMPRESSION_TYPE,
LogAggregationService.DEFAULT_COMPRESSION_TYPE), null, conf);
YarnConfiguration.NM_LOG_AGG_COMPRESSION_TYPE,
YarnConfiguration.DEFAULT_NM_LOG_AGG_COMPRESSION_TYPE), null, conf);
}
public void append(LogKey logKey, LogValue logValue) throws IOException {

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_BIND_ADDRESS;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_BIND_ADDRESS;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
@ -43,12 +40,9 @@ import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorAppFinishedEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorAppStartedEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorContainerFinishedEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.event.LogAggregatorEvent;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.ConverterUtils;
@ -65,13 +59,6 @@ public class LogAggregationService extends AbstractService implements
Path remoteRootLogDir;
private String nodeFile;
static final String LOG_COMPRESSION_TYPE = NMConfig.NM_PREFIX
+ "logaggregation.log_compression_type";
static final String DEFAULT_COMPRESSION_TYPE = "none";
private static final String LOG_RENTENTION_POLICY_CONFIG_KEY =
NMConfig.NM_PREFIX + "logaggregation.retain-policy";
private final ConcurrentMap<ApplicationId, AppLogAggregator> appLogAggregators;
private final ExecutorService threadPool;
@ -86,17 +73,17 @@ public class LogAggregationService extends AbstractService implements
public synchronized void init(Configuration conf) {
this.localRootLogDirs =
conf.getStrings(NMConfig.NM_LOG_DIR, NMConfig.DEFAULT_NM_LOG_DIR);
conf.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS);
this.remoteRootLogDir =
new Path(conf.get(NMConfig.REMOTE_USER_LOG_DIR,
NMConfig.DEFAULT_REMOTE_APP_LOG_DIR));
new Path(conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
super.init(conf);
}
@Override
public synchronized void start() {
String address =
getConfig().get(NM_BIND_ADDRESS, DEFAULT_NM_BIND_ADDRESS);
getConfig().get(YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS);
InetSocketAddress cmBindAddress = NetUtils.createSocketAddr(address);
try {
this.nodeFile =

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogReader;
import org.apache.hadoop.yarn.util.ConverterUtils;
@ -106,8 +105,8 @@ public class LogDumper extends Configured implements Tool {
return -1;
} else {
Path remoteRootLogDir =
new Path(getConf().get(NMConfig.REMOTE_USER_LOG_DIR,
NMConfig.DEFAULT_REMOTE_APP_LOG_DIR));
new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
AggregatedLogFormat.LogReader reader =
new AggregatedLogFormat.LogReader(getConf(),
LogAggregationService.getRemoteNodeLogFileForApp(
@ -151,8 +150,8 @@ public class LogDumper extends Configured implements Tool {
dumpAllContainersLogs(ApplicationId appId, DataOutputStream out)
throws IOException {
Path remoteRootLogDir =
new Path(getConf().get(NMConfig.REMOTE_USER_LOG_DIR,
NMConfig.DEFAULT_REMOTE_APP_LOG_DIR));
new Path(getConf().get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
Path remoteAppLogDir =
LogAggregationService.getRemoteAppLogDir(remoteRootLogDir, appId);
RemoteIterator<FileStatus> nodeFiles =

View File

@ -29,11 +29,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
@ -45,14 +45,6 @@ public class ContainersMonitorImpl extends AbstractService implements
final static Log LOG = LogFactory
.getLog(ContainersMonitorImpl.class);
private final static String MONITORING_INTERVAL_CONFIG_KEY =
NMConfig.NM_PREFIX + "containers-monitor.monitoring-interval";
public static final String RESOURCE_CALCULATOR_PLUGIN_CONFIG_KEY =
NMConfig.NM_PREFIX + "containers-monitor.resourcecalculatorplugin";
public static final String NM_RESERVED_PHYSICALMEMORY_MB =
NMConfig.NM_PREFIX + "reserved-physical-memory.mb";
private final static int MONITORING_INTERVAL_DEFAULT = 3000;
private long monitoringInterval;
private MonitoringThread monitoringThread;
@ -96,11 +88,11 @@ public class ContainersMonitorImpl extends AbstractService implements
@Override
public synchronized void init(Configuration conf) {
this.monitoringInterval =
conf.getLong(MONITORING_INTERVAL_CONFIG_KEY,
MONITORING_INTERVAL_DEFAULT);
conf.getLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS,
YarnConfiguration.DEFAULT_NM_CONTAINER_MON_INTERVAL_MS);
Class<? extends ResourceCalculatorPlugin> clazz =
conf.getClass(RESOURCE_CALCULATOR_PLUGIN_CONFIG_KEY, null,
conf.getClass(YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR, null,
ResourceCalculatorPlugin.class);
this.resourceCalculatorPlugin =
ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, conf);
@ -120,7 +112,7 @@ public class ContainersMonitorImpl extends AbstractService implements
// ///////// Virtual memory configuration //////
this.maxVmemAllottedForContainers =
conf.getLong(NMConfig.NM_VMEM_GB, NMConfig.DEFAULT_NM_VMEM_GB);
conf.getLong(YarnConfiguration.NM_VMEM_GB, YarnConfiguration.DEFAULT_NM_VMEM_GB);
this.maxVmemAllottedForContainers =
this.maxVmemAllottedForContainers * 1024 * 1024 * 1024L; //Normalize
@ -131,7 +123,7 @@ public class ContainersMonitorImpl extends AbstractService implements
// ///////// Physical memory configuration //////
long reservedPmemOnNM =
conf.getLong(NM_RESERVED_PHYSICALMEMORY_MB, DISABLED_MEMORY_LIMIT);
conf.getLong(YarnConfiguration.NM_RESERVED_MEMORY_MB, DISABLED_MEMORY_LIMIT);
reservedPmemOnNM =
reservedPmemOnNM == DISABLED_MEMORY_LIMIT
? DISABLED_MEMORY_LIMIT

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.yarn.server.nodemanager.webapp;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.DEFAULT_NM_LOG_DIR;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOG_DIR;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
@ -33,10 +30,10 @@ import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
import org.apache.hadoop.yarn.util.ConverterUtils;
@ -64,7 +61,7 @@ public class ContainerLogsPage extends NMView {
@Inject
public ContainersLogsBlock(Configuration conf, Context context) {
this.conf = conf;
this.logsSelector = new LocalDirAllocator(NMConfig.NM_LOG_DIR);
this.logsSelector = new LocalDirAllocator(YarnConfiguration.NM_LOG_DIRS);
this.nmContext = context;
this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
}
@ -176,7 +173,7 @@ public class ContainerLogsPage extends NMView {
static List<File>
getContainerLogDirs(Configuration conf, ContainerId containerId) {
String[] logDirs =
conf.getStrings(NM_LOG_DIR, DEFAULT_NM_LOG_DIR);
conf.getStrings(YarnConfiguration.NM_LOG_DIRS, YarnConfiguration.DEFAULT_NM_LOG_DIRS);
List<File> containerLogDirs = new ArrayList<File>(logDirs.length);
for (String logDir : logDirs) {
String appIdStr = ConverterUtils.toString(containerId.getAppId());

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp;
import static org.apache.hadoop.yarn.util.StringHelper.join;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.webapp.Controller;
import com.google.inject.Inject;

View File

@ -24,8 +24,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.webapp.WebApp;
@ -52,8 +52,8 @@ public class WebServer extends AbstractService {
@Override
public synchronized void start() {
String bindAddress = getConfig().get(NMConfig.NM_HTTP_BIND_ADDRESS,
NMConfig.DEFAULT_NM_HTTP_BIND_ADDRESS);
String bindAddress = getConfig().get(YarnConfiguration.NM_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS);
LOG.info("Instantiating NMWebApp at " + bindAddress);
try {
this.webApp =

View File

@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
import org.junit.After;
@ -119,13 +120,13 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
private boolean shouldRunTest() {
return System
.getProperty(LinuxContainerExecutor.CONTAINER_EXECUTOR_EXEC_KEY) != null;
.getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;
}
@Override
protected ContainerExecutor createContainerExecutor() {
super.conf.set(LinuxContainerExecutor.CONTAINER_EXECUTOR_EXEC_KEY, System
.getProperty(LinuxContainerExecutor.CONTAINER_EXECUTOR_EXEC_KEY));
super.conf.set(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH, System
.getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH));
LinuxContainerExecutor linuxContainerExecutor =
new LinuxContainerExecutor();
linuxContainerExecutor.setConf(super.conf);

View File

@ -74,9 +74,9 @@ public class TestEventFlow {
Context context = new NMContext();
YarnConfiguration conf = new YarnConfiguration();
conf.set(NMConfig.NM_LOCAL_DIR, localDir.getAbsolutePath());
conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath());
conf.set(NMConfig.REMOTE_USER_LOG_DIR, remoteLogDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
ContainerExecutor exec = new DefaultContainerExecutor();
exec.setConf(conf);

View File

@ -207,13 +207,13 @@ public class TestNodeStatusUpdater {
};
YarnConfiguration conf = new YarnConfiguration();
conf.setInt(NMConfig.NM_VMEM_GB, 5); // 5GB
conf.set(NMConfig.NM_BIND_ADDRESS, "127.0.0.1:12345");
conf.set(NMConfig.NM_LOCALIZER_BIND_ADDRESS, "127.0.0.1:12346");
conf.set(NMConfig.NM_LOG_DIR, new Path(basedir, "logs").toUri().getPath());
conf.set(NMConfig.REMOTE_USER_LOG_DIR, new Path(basedir, "remotelogs")
conf.setInt(YarnConfiguration.NM_VMEM_GB, 5); // 5GB
conf.set(YarnConfiguration.NM_ADDRESS, "127.0.0.1:12345");
conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "127.0.0.1:12346");
conf.set(YarnConfiguration.NM_LOG_DIRS, new Path(basedir, "logs").toUri().getPath());
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, new Path(basedir, "remotelogs")
.toUri().getPath());
conf.set(NMConfig.NM_LOCAL_DIR, new Path(basedir, "nm0").toUri().getPath());
conf.set(YarnConfiguration.NM_LOCAL_DIRS, new Path(basedir, "nm0").toUri().getPath());
nm.init(conf);
new Thread() {
public void run() {

View File

@ -22,6 +22,7 @@ import java.net.InetSocketAddress;
import org.apache.avro.ipc.Server;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
@ -39,7 +40,7 @@ public class TestPBLocalizerRPC {
static RecordFactory createPBRecordFactory() {
Configuration conf = new Configuration();
conf.set(RecordFactoryProvider.RPC_SERIALIZER_KEY, "protocolbuffers");
conf.set(YarnConfiguration.IPC_SERIALIZER_TYPE, "protocolbuffers");
return RecordFactoryProvider.getRecordFactory(conf);
}

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.RPCUtil;
@ -46,7 +47,7 @@ public class TestPBRecordImpl {
static RecordFactory createPBRecordFactory() {
Configuration conf = new Configuration();
conf.set(RecordFactoryProvider.RPC_SERIALIZER_KEY, "protocolbuffers");
conf.set(YarnConfiguration.IPC_SERIALIZER_TYPE, "protocolbuffers");
return RecordFactoryProvider.getRecordFactory(conf);
}

View File

@ -47,7 +47,6 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.LocalRMInterface;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
@ -129,10 +128,10 @@ public abstract class BaseContainerManagerTest {
LOG.info("Created tmpDir in " + tmpDir.getAbsolutePath());
String bindAddress = "0.0.0.0:5555";
conf.set(NMConfig.NM_BIND_ADDRESS, bindAddress);
conf.set(NMConfig.NM_LOCAL_DIR, localDir.getAbsolutePath());
conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath());
conf.set(NMConfig.REMOTE_USER_LOG_DIR, remoteLogDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_ADDRESS, bindAddress);
conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
// Default delSrvc
delSrvc = new DeletionService(exec) {

View File

@ -27,6 +27,7 @@ import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
@ -95,10 +96,10 @@ public class TestAuxServices {
@Test
public void testAuxEventDispatch() {
Configuration conf = new Configuration();
conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"),
conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"),
ServiceA.class, Service.class);
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"),
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"),
ServiceB.class, Service.class);
conf.setInt("A.expected.init", 1);
conf.setInt("B.expected.stop", 1);
@ -123,10 +124,10 @@ public class TestAuxServices {
@Test
public void testAuxServices() {
Configuration conf = new Configuration();
conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"),
conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"),
ServiceA.class, Service.class);
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"),
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"),
ServiceB.class, Service.class);
final AuxServices aux = new AuxServices();
aux.init(conf);
@ -154,10 +155,10 @@ public class TestAuxServices {
@Test
public void testAuxServicesMeta() {
Configuration conf = new Configuration();
conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"),
conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"),
ServiceA.class, Service.class);
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"),
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"),
ServiceB.class, Service.class);
final AuxServices aux = new AuxServices();
aux.init(conf);
@ -191,10 +192,10 @@ public class TestAuxServices {
@Test
public void testAuxUnexpectedStop() {
Configuration conf = new Configuration();
conf.setStrings(AuxServices.AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Asrv"),
conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, new String[] { "Asrv", "Bsrv" });
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Asrv"),
ServiceA.class, Service.class);
conf.setClass(String.format(AuxServices.AUX_SERVICE_CLASS_FMT, "Bsrv"),
conf.setClass(String.format(YarnConfiguration.NM_AUX_SERVICE_FMT, "Bsrv"),
ServiceB.class, Service.class);
final AuxServices aux = new AuxServices();
aux.init(conf);

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
@ -74,8 +75,6 @@ import static org.junit.Assert.*;
import org.mockito.ArgumentMatcher;
import static org.mockito.Mockito.*;
import static org.apache.hadoop.yarn.server.nodemanager.NMConfig.NM_LOCAL_DIR;
public class TestResourceLocalizationService {
static final Path basedir =
@ -110,7 +109,7 @@ public class TestResourceLocalizationService {
localDirs.add(lfs.makeQualified(new Path(basedir, i + "")));
sDirs[i] = localDirs.get(i).toString();
}
conf.setStrings(NM_LOCAL_DIR, sDirs);
conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs);
// initialize ResourceLocalizationService
locService.init(conf);
@ -149,7 +148,7 @@ public class TestResourceLocalizationService {
localDirs.add(lfs.makeQualified(new Path(basedir, i + "")));
sDirs[i] = localDirs.get(i).toString();
}
conf.setStrings(NM_LOCAL_DIR, sDirs);
conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs);
Server ignore = mock(Server.class);
DrainDispatcher dispatcher = new DrainDispatcher();

View File

@ -47,11 +47,11 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogKey;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AggregatedLogFormat.LogReader;
@ -92,8 +92,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
public void testLocalFileDeletionAfterUpload() throws IOException {
this.delSrvc = new DeletionService(createContainerExecutor());
this.delSrvc.init(conf);
this.conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath());
this.conf.set(NMConfig.REMOTE_USER_LOG_DIR,
this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
this.remoteRootLogDir.getAbsolutePath());
LogAggregationService logAggregationService =
new LogAggregationService(this.delSrvc);
@ -140,8 +140,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
@Test
public void testNoContainerOnNode() {
this.conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath());
this.conf.set(NMConfig.REMOTE_USER_LOG_DIR,
this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
this.remoteRootLogDir.getAbsolutePath());
LogAggregationService logAggregationService =
new LogAggregationService(this.delSrvc);
@ -173,8 +173,8 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
@Test
public void testMultipleAppsLogAggregation() throws IOException {
this.conf.set(NMConfig.NM_LOG_DIR, localLogDir.getAbsolutePath());
this.conf.set(NMConfig.REMOTE_USER_LOG_DIR,
this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
this.remoteRootLogDir.getAbsolutePath());
LogAggregationService logAggregationService =
new LogAggregationService(this.delSrvc);

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
@ -70,7 +71,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
@Before
public void setup() throws IOException {
conf.setClass(
ContainersMonitorImpl.RESOURCE_CALCULATOR_PLUGIN_CONFIG_KEY,
YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
LinuxResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
super.setup();
}

View File

@ -28,12 +28,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.NMConfig;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@ -72,7 +72,7 @@ public class TestNMWebServer {
};
WebServer server = new WebServer(nmContext, resourceView);
Configuration conf = new Configuration();
conf.set(NMConfig.NM_LOCAL_DIR, testRootDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
server.init(conf);
server.start();

View File

@ -43,7 +43,7 @@ public class AdminSecurityInfo extends SecurityInfo {
@Override
public String serverPrincipal() {
return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY;
return YarnConfiguration.RM_PRINCIPAL;
}
@Override

View File

@ -83,12 +83,12 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
public void init(Configuration conf) {
super.init(conf);
String bindAddress =
conf.get(RMConfig.ADMIN_ADDRESS,
RMConfig.DEFAULT_ADMIN_BIND_ADDRESS);
conf.get(YarnConfiguration.RM_ADMIN_ADDRESS,
YarnConfiguration.RM_ADMIN_ADDRESS);
masterServiceAddress = NetUtils.createSocketAddr(bindAddress);
adminAcl =
new AccessControlList(
conf.get(RMConfig.RM_ADMIN_ACL, RMConfig.DEFAULT_RM_ADMIN_ACL));
conf.get(YarnConfiguration.RM_ADMIN_ACL, YarnConfiguration.DEFAULT_RM_ADMIN_ACL));
}
public void start() {
@ -100,8 +100,8 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
this.server =
rpc.getServer(RMAdminProtocol.class, this, masterServiceAddress,
serverConf, null,
serverConf.getInt(RMConfig.RM_ADMIN_THREADS,
RMConfig.DEFAULT_RM_ADMIN_THREADS));
serverConf.getInt(YarnConfiguration.RM_ADMIN_CLIENT_THREAD_COUNT,
YarnConfiguration.DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT));
this.server.start();
super.start();
}
@ -219,7 +219,7 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
Configuration conf = new Configuration();
adminAcl =
new AccessControlList(
conf.get(RMConfig.RM_ADMIN_ACL, RMConfig.DEFAULT_RM_ADMIN_ACL));
conf.get(YarnConfiguration.RM_ADMIN_ACL, YarnConfiguration.DEFAULT_RM_ADMIN_ACL));
RMAuditLogger.logSuccess(user.getShortUserName(), "refreshAdminAcls",
"AdminService");

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@InterfaceAudience.Private
public class ApplicationACLsManager {
@ -36,7 +37,8 @@ public class ApplicationACLsManager {
}
public boolean areACLsEnabled() {
return conf.getBoolean(RMConfig.RM_ACLS_ENABLED, false);
return conf.getBoolean(YarnConfiguration.RM_ACL_ENABLE,
YarnConfiguration.DEFAULT_RM_ACL_ENABLE);
}
/**

View File

@ -90,8 +90,8 @@ public class ApplicationMasterService extends AbstractService implements
@Override
public void init(Configuration conf) {
String bindAddress =
conf.get(YarnConfiguration.SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS);
conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS);
masterServiceAddress = NetUtils.createSocketAddr(bindAddress);
super.init(conf);
}
@ -105,8 +105,8 @@ public class ApplicationMasterService extends AbstractService implements
this.server =
rpc.getServer(AMRMProtocol.class, this, masterServiceAddress,
serverConf, this.appTokenManager,
serverConf.getInt(RMConfig.RM_AM_THREADS,
RMConfig.DEFAULT_RM_AM_THREADS));
serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT));
this.server.start();
super.start();
}

View File

@ -31,7 +31,6 @@ import org.apache.avro.ipc.Server;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.UserGroupInformation;
@ -58,10 +57,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@ -74,7 +71,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstant
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@ -114,8 +110,8 @@ public class ClientRMService extends AbstractService implements
@Override
public void init(Configuration conf) {
clientServiceBindAddress =
conf.get(YarnConfiguration.APPSMANAGER_ADDRESS,
YarnConfiguration.DEFAULT_APPSMANAGER_BIND_ADDRESS);
conf.get(YarnConfiguration.RM_ADDRESS,
YarnConfiguration.DEFAULT_RM_ADDRESS);
clientBindAddress =
NetUtils.createSocketAddr(clientServiceBindAddress);
@ -138,8 +134,8 @@ public class ClientRMService extends AbstractService implements
rpc.getServer(ClientRMProtocol.class, this,
clientBindAddress,
clientServerConf, null,
clientServerConf.getInt(RMConfig.RM_CLIENT_THREADS,
RMConfig.DEFAULT_RM_CLIENT_THREADS));
clientServerConf.getInt(YarnConfiguration.RM_CLIENT_THREAD_COUNT,
YarnConfiguration.DEFAULT_RM_CLIENT_THREAD_COUNT));
this.server.start();
super.start();
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.SystemClock;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
@ -38,11 +39,11 @@ public class NMLivelinessMonitor extends AbstractLivelinessMonitor<NodeId> {
public void init(Configuration conf) {
super.init(conf);
setExpireInterval(conf.getInt(RMConfig.NM_EXPIRY_INTERVAL,
RMConfig.DEFAULT_NM_EXPIRY_INTERVAL));
setExpireInterval(conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS));
setMonitorInterval(conf.getInt(
RMConfig.NMLIVELINESS_MONITORING_INTERVAL,
RMConfig.DEFAULT_NMLIVELINESS_MONITORING_INTERVAL));
YarnConfiguration.RM_NM_LIVENESS_MONITOR_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_NM_LIVENESS_MONITOR_INTERVAL_MS));
}
@Override

View File

@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.HostsFileReader;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.service.AbstractService;
public class NodesListManager extends AbstractService{
@ -48,18 +49,18 @@ public class NodesListManager extends AbstractService{
try {
this.hostsReader =
new HostsFileReader(
conf.get(RMConfig.RM_NODES_INCLUDE_FILE,
RMConfig.DEFAULT_RM_NODES_INCLUDE_FILE),
conf.get(RMConfig.RM_NODES_EXCLUDE_FILE,
RMConfig.DEFAULT_RM_NODES_EXCLUDE_FILE)
conf.get(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
YarnConfiguration.DEFAULT_RM_NODES_INCLUDE_FILE_PATH),
conf.get(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH)
);
printConfiguredHosts();
} catch (IOException ioe) {
LOG.warn("Failed to init hostsReader, disabling", ioe);
try {
this.hostsReader =
new HostsFileReader(RMConfig.DEFAULT_RM_NODES_INCLUDE_FILE,
RMConfig.DEFAULT_RM_NODES_EXCLUDE_FILE);
new HostsFileReader(YarnConfiguration.DEFAULT_RM_NODES_INCLUDE_FILE_PATH,
YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH);
} catch (IOException ioe2) {
// Should *never* happen
this.hostsReader = null;
@ -74,10 +75,10 @@ public class NodesListManager extends AbstractService{
return;
}
LOG.debug("hostsReader: in=" + conf.get(RMConfig.RM_NODES_INCLUDE_FILE,
RMConfig.DEFAULT_RM_NODES_INCLUDE_FILE) + " out=" +
conf.get(RMConfig.RM_NODES_EXCLUDE_FILE,
RMConfig.DEFAULT_RM_NODES_EXCLUDE_FILE));
LOG.debug("hostsReader: in=" + conf.get(YarnConfiguration.RM_NODES_INCLUDE_FILE_PATH,
YarnConfiguration.DEFAULT_RM_NODES_INCLUDE_FILE_PATH) + " out=" +
conf.get(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH));
for (String include : hostsReader.getHosts()) {
LOG.debug("include: " + include);
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
@ -48,7 +49,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
private static final Log LOG = LogFactory.getLog(RMAppManager.class);
private int completedAppsMax = RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX;
private int completedAppsMax = YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS;
private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
private final RMContext rmContext;
@ -66,8 +67,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
this.masterService = masterService;
this.conf = conf;
setCompletedAppsMax(conf.getInt(
RMConfig.EXPIRE_APPLICATIONS_COMPLETED_MAX,
RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX));
YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS,
YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS));
}
/**

View File

@ -1,93 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
public class RMConfig {
public static final String RM_KEYTAB = YarnConfiguration.RM_PREFIX
+ "keytab";
public static final String ZK_ADDRESS = YarnConfiguration.RM_PREFIX
+ "zookeeper.address";
public static final String ZK_SESSION_TIMEOUT = YarnConfiguration.RM_PREFIX
+ "zookeeper.session.timeout";
public static final String ADMIN_ADDRESS = YarnConfiguration.RM_PREFIX
+ "admin.address";
public static final String AM_MAX_RETRIES = YarnConfiguration.RM_PREFIX
+ "application.max.retries";
public static final int DEFAULT_ZK_TIMEOUT = 60000;
public static final int DEFAULT_AM_MAX_RETRIES = 3;
public static final int DEFAULT_AM_EXPIRY_INTERVAL = 600000;
public static final String NM_EXPIRY_INTERVAL = YarnConfiguration.RM_PREFIX
+ "nodemanager.expiry.interval";
public static final int DEFAULT_NM_EXPIRY_INTERVAL = 600000;
public static final String DEFAULT_ADMIN_BIND_ADDRESS = "0.0.0.0:8141";
public static final String RESOURCE_SCHEDULER = YarnConfiguration.RM_PREFIX
+ "scheduler";
public static final String RM_STORE = YarnConfiguration.RM_PREFIX + "store";
public static final String AMLIVELINESS_MONITORING_INTERVAL =
YarnConfiguration.RM_PREFIX
+ "amliveliness-monitor.monitoring-interval";
public static final int DEFAULT_AMLIVELINESS_MONITORING_INTERVAL = 1000;
public static final String CONTAINER_LIVELINESS_MONITORING_INTERVAL
= YarnConfiguration.RM_PREFIX
+ "amliveliness-monitor.monitoring-interval";
public static final int DEFAULT_CONTAINER_LIVELINESS_MONITORING_INTERVAL = 600000;
public static final String NMLIVELINESS_MONITORING_INTERVAL =
YarnConfiguration.RM_PREFIX
+ "nmliveliness-monitor.monitoring-interval";
public static final int DEFAULT_NMLIVELINESS_MONITORING_INTERVAL = 1000;
public static final String RM_RESOURCE_TRACKER_THREADS =
YarnConfiguration.RM_PREFIX + "resource.tracker.threads";
public static final int DEFAULT_RM_RESOURCE_TRACKER_THREADS = 10;
public static final String RM_CLIENT_THREADS =
YarnConfiguration.RM_PREFIX + "client.threads";
public static final int DEFAULT_RM_CLIENT_THREADS = 10;
public static final String RM_AM_THREADS =
YarnConfiguration.RM_PREFIX + "am.threads";
public static final int DEFAULT_RM_AM_THREADS = 10;
public static final String RM_ADMIN_THREADS =
YarnConfiguration.RM_PREFIX + "admin.threads";
public static final int DEFAULT_RM_ADMIN_THREADS = 1;
/* key for looking up the acls configuration for acls checking for application */
public static final String RM_ACLS_ENABLED = YarnConfiguration.RM_PREFIX +
"acls.enabled";
public static final String RM_ADMIN_ACL =
YarnConfiguration.RM_PREFIX + "admin.acl";
public static final String DEFAULT_RM_ADMIN_ACL = "*";
public static final String RM_NODES_INCLUDE_FILE =
YarnConfiguration.RM_PREFIX + "nodes.include";
public static final String DEFAULT_RM_NODES_INCLUDE_FILE = "";
public static final String RM_NODES_EXCLUDE_FILE =
YarnConfiguration.RM_PREFIX + "nodes.exclude";
public static final String DEFAULT_RM_NODES_EXCLUDE_FILE = "";
// the maximum number of completed applications RM keeps
public static final String EXPIRE_APPLICATIONS_COMPLETED_MAX =
YarnConfiguration.RM_PREFIX + "expire.applications.completed.max";
public static final int DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX = 10000;
}

View File

@ -202,7 +202,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
protected ResourceScheduler createScheduler() {
return
ReflectionUtils.newInstance(
conf.getClass(RMConfig.RESOURCE_SCHEDULER,
conf.getClass(YarnConfiguration.RM_SCHEDULER,
FifoScheduler.class, ResourceScheduler.class),
this.conf);
}
@ -384,8 +384,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
protected void startWepApp() {
webApp = WebApps.$for("yarn", masterService).at(
conf.get(YarnConfiguration.RM_WEBAPP_BIND_ADDRESS,
YarnConfiguration.DEFAULT_RM_WEBAPP_BIND_ADDRESS)).
conf.get(YarnConfiguration.RM_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS)).
start(new RMWebApp(this));
}
@ -415,8 +415,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
}
protected void doSecureLogin() throws IOException {
SecurityUtil.login(conf, RMConfig.RM_KEYTAB,
YarnConfiguration.RM_SERVER_PRINCIPAL_KEY);
SecurityUtil.login(conf, YarnConfiguration.RM_KEYTAB,
YarnConfiguration.RM_PRINCIPAL);
}
@Override

View File

@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.RMNMSecurityInfoClass;
import org.apache.hadoop.yarn.server.YarnServerConfig;
import org.apache.hadoop.yarn.server.api.ResourceTracker;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@ -102,8 +101,8 @@ public class ResourceTrackerService extends AbstractService implements
@Override
public synchronized void init(Configuration conf) {
String resourceTrackerBindAddress =
conf.get(YarnServerConfig.RESOURCETRACKER_ADDRESS,
YarnServerConfig.DEFAULT_RESOURCETRACKER_BIND_ADDRESS);
conf.get(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS);
resourceTrackerAddress = NetUtils.createSocketAddr(resourceTrackerBindAddress);
RackResolver.init(conf);
@ -123,8 +122,8 @@ public class ResourceTrackerService extends AbstractService implements
this.server =
rpc.getServer(ResourceTracker.class, this, resourceTrackerAddress,
rtServerConf, null,
rtServerConf.getInt(RMConfig.RM_RESOURCE_TRACKER_THREADS,
RMConfig.DEFAULT_RM_RESOURCE_TRACKER_THREADS));
rtServerConf.getInt(YarnConfiguration.RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT,
YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT));
this.server.start();
}

View File

@ -33,7 +33,6 @@ import org.apache.commons.codec.binary.Base64;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.io.DataInputByteBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Text;
@ -225,8 +224,8 @@ public class AMLauncher implements Runnable {
new Token<ApplicationTokenIdentifier>(id,
this.applicationTokenSecretManager);
String schedulerAddressStr =
this.conf.get(YarnConfiguration.SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS);
this.conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS);
InetSocketAddress unresolvedAddr =
NetUtils.createSocketAddr(schedulerAddressStr);
String resolvedAddr =

View File

@ -23,14 +23,14 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
public class StoreFactory {
public static Store getStore(Configuration conf) {
Store store = ReflectionUtils.newInstance(
conf.getClass(RMConfig.RM_STORE,
conf.getClass(YarnConfiguration.RM_STORE,
MemStore.class, Store.class),
conf);
return store;

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPB
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationMasterProto;
@ -48,7 +49,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
@ -91,9 +91,9 @@ public class ZKStore implements Store {
public ZKStore(Configuration conf) throws IOException {
this.conf = conf;
this.ZK_ADDRESS = conf.get(RMConfig.ZK_ADDRESS);
this.ZK_TIMEOUT = conf.getInt(RMConfig.ZK_SESSION_TIMEOUT,
RMConfig.DEFAULT_ZK_TIMEOUT);
this.ZK_ADDRESS = conf.get(YarnConfiguration.RM_ZK_STORE_ADDRESS);
this.ZK_TIMEOUT = conf.getInt(YarnConfiguration.RM_ZK_STORE_TIMEOUT_MS,
YarnConfiguration.DEFAULT_RM_ZK_STORE_TIMEOUT_MS);
zkClient = new ZooKeeper(this.ZK_ADDRESS,
this.ZK_TIMEOUT,
createZKWatcher()

View File

@ -37,9 +37,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
@ -181,8 +181,8 @@ public class RMAppImpl implements RMApp {
this.masterService = masterService;
this.startTime = System.currentTimeMillis();
this.maxRetries = conf.getInt(RMConfig.AM_MAX_RETRIES,
RMConfig.DEFAULT_AM_MAX_RETRIES);
this.maxRetries = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES);
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
this.readLock = lock.readLock();

View File

@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
public class AMLivelinessMonitor extends AbstractLivelinessMonitor<ApplicationAttemptId> {
@ -38,10 +37,10 @@ public class AMLivelinessMonitor extends AbstractLivelinessMonitor<ApplicationAt
public void init(Configuration conf) {
super.init(conf);
setExpireInterval(conf.getInt(YarnConfiguration.AM_EXPIRY_INTERVAL,
RMConfig.DEFAULT_AM_EXPIRY_INTERVAL));
setMonitorInterval(conf.getInt(RMConfig.AMLIVELINESS_MONITORING_INTERVAL,
RMConfig.DEFAULT_AMLIVELINESS_MONITORING_INTERVAL));
setExpireInterval(conf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS));
setMonitorInterval(conf.getInt(YarnConfiguration.RM_AM_LIVENESS_MONITOR_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_AM_LIVENESS_MONITOR_INTERVAL_MS));
}
@Override

View File

@ -21,9 +21,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.SystemClock;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
@ -40,10 +40,10 @@ public class ContainerAllocationExpirer extends
public void init(Configuration conf) {
super.init(conf);
setExpireInterval(conf.getInt(
RMConfig.CONTAINER_LIVELINESS_MONITORING_INTERVAL,
RMConfig.DEFAULT_CONTAINER_LIVELINESS_MONITORING_INTERVAL));
setMonitorInterval(conf.getInt(RMConfig.AMLIVELINESS_MONITORING_INTERVAL,
RMConfig.DEFAULT_AMLIVELINESS_MONITORING_INTERVAL));
YarnConfiguration.RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_CONTAINER_LIVENESS_MONITOR_INTERVAL_MS));
setMonitorInterval(conf.getInt(YarnConfiguration.RM_AM_LIVENESS_MONITOR_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_AM_LIVENESS_MONITOR_INTERVAL_MS));
}
@Override

View File

@ -40,7 +40,7 @@ public class CapacitySchedulerConfiguration extends Configuration {
private static final String CS_CONFIGURATION_FILE = "capacity-scheduler.xml";
@Private
public static final String PREFIX = "yarn.capacity-scheduler.";
public static final String PREFIX = "yarn.scheduler.capacity.";
@Private
public static final String DOT = ".";

View File

@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@ -67,7 +66,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@ -105,8 +103,7 @@ public class FifoScheduler implements ResourceScheduler {
private static final int MINIMUM_MEMORY = 1024;
private static final String FIFO_PREFIX =
YarnConfiguration.RM_PREFIX + "fifo.";
private static final String FIFO_PREFIX = "yarn.scheduler.fifo.";
@Private
public static final String MINIMUM_ALLOCATION =
FIFO_PREFIX + "minimum-allocation-mb";

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.admin.AdminSecurityInfo;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.apache.hadoop.yarn.server.resourcemanager.api.RMAdminProtocol;
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshAdminAclsRequest;
import org.apache.hadoop.yarn.server.resourcemanager.api.protocolrecords.RefreshNodesRequest;
@ -146,8 +145,8 @@ public class RMAdmin extends Configured implements Tool {
// Create the client
final String adminAddress =
conf.get(RMConfig.ADMIN_ADDRESS,
RMConfig.DEFAULT_ADMIN_BIND_ADDRESS);
conf.get(YarnConfiguration.RM_ADMIN_ADDRESS,
YarnConfiguration.RM_ADMIN_ADDRESS);
final YarnRPC rpc = YarnRPC.create(conf);
if (UserGroupInformation.isSecurityEnabled()) {

View File

@ -1,58 +1,79 @@
<configuration>
<property>
<name>yarn.capacity-scheduler.maximum-applications</name>
<name>yarn.scheduler.capacity.maximum-applications</name>
<value>10000</value>
<description>Maximum number of applications that can be running.
</description>
</property>
<property>
<name>yarn.capacity-scheduler.maximum-am-resource-percent</name>
<name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
<value>0.1</value>
</property>
<property>
<name>yarn.capacity-scheduler.root.queues</name>
<name>yarn.scheduler.capacity.root.queues</name>
<value>default</value>
<description>The queues at the this level (root is the root queue).
</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.capacity</name>
<name>yarn.scheduler.capacity.root.capacity</name>
<value>100</value>
<description>The total capacity as a percentage out of 100 for this queue.
If it has child queues then this includes their capacity as well.
The child queues capacity should add up to their parent queue's capacity
or less.</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.acl_administer_queues</name>
<name>yarn.scheduler.capacity.root.acl_administer_queues</name>
<value>*</value>
<description>The ACL for who can administer this queue. i.e.
change sub queue allocations.</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.default.capacity</name>
<name>yarn.scheduler.capacity.root.default.capacity</name>
<value>100</value>
<description>default queue target capacity.</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.default.user-limit-factor</name>
<name>yarn.scheduler.capacity.root.default.user-limit-factor</name>
<value>1</value>
<description>default queue user limit a percantage from 0.0 to 1.0.
</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.default.maximum-capacity</name>
<name>yarn.scheduler.capacity.root.default.maximum-capacity</name>
<value>-1</value>
<description>the maximum capacity of the default queue -1 disables.
</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.default.state</name>
<name>yarn.scheduler.capacity.root.default.state</name>
<value>RUNNING</value>
<description>The state of the default queue. can be RUNNING or STOPPED
</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.default.acl_submit_jobs</name>
<name>yarn.scheduler.capacity.root.default.acl_submit_jobs</name>
<value>*</value>
<description>The ACL of who can submit jobs to the default queue.
</description>
</property>
<property>
<name>yarn.capacity-scheduler.root.default.acl_administer_jobs</name>
<name>yarn.scheduler.capacity.root.default.acl_administer_jobs</name>
<value>*</value>
<description>The ACL of who can administer jobs on the default queue.
</description>
</property>
</configuration>

View File

@ -32,6 +32,7 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -47,7 +48,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.RMAppManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMConfig;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
@ -153,14 +153,14 @@ public class TestAppManager{
public TestRMAppManager(RMContext context, Configuration conf) {
super(context, null, null, null, conf);
setCompletedAppsMax(RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX);
setCompletedAppsMax(YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS);
}
public TestRMAppManager(RMContext context, ClientToAMSecretManager
clientToAMSecretManager, YarnScheduler scheduler,
ApplicationMasterService masterService, Configuration conf) {
super(context, clientToAMSecretManager, scheduler, masterService, conf);
setCompletedAppsMax(RMConfig.DEFAULT_EXPIRE_APPLICATIONS_COMPLETED_MAX);
setCompletedAppsMax(YarnConfiguration.DEFAULT_RM_MAX_COMPLETED_APPLICATIONS);
}
public void checkAppNumCompletedLimit() {

Some files were not shown because too many files have changed in this diff Show More