MAPREDUCE-2864. Normalize configuration variable names for YARN. Contributed by Robert Evans.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1166955 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
96a5bef79e
commit
fafe8cd28e
@ -252,6 +252,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
|
||||
|
@ -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>
|
||||
|
||||
|
@ -42,7 +42,6 @@
|
||||
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;
|
||||
|
@ -34,10 +34,10 @@
|
||||
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 @@ protected void startRpcServer() {
|
||||
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();
|
||||
|
@ -20,9 +20,6 @@
|
||||
|
||||
// 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;
|
||||
|
@ -37,12 +37,12 @@
|
||||
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 void init(Configuration conf) {
|
||||
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 void init(Configuration conf) {
|
||||
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);
|
||||
|
@ -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";
|
||||
|
||||
}
|
@ -154,7 +154,7 @@ public void init(final Configuration conf) {
|
||||
// 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 @@ protected Speculator createSpeculator(Configuration conf, AppContext context) {
|
||||
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 @@ protected Speculator createSpeculator(Configuration conf, AppContext context) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
@ -32,6 +32,7 @@
|
||||
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.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 void start() {
|
||||
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()
|
||||
|
@ -94,7 +94,6 @@
|
||||
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 @@ private WrappedProgressSplitsBlock getProgressSplitBlock() {
|
||||
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 {
|
||||
|
@ -33,8 +33,8 @@
|
||||
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 synchronized void init(Configuration conf) {
|
||||
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.
|
||||
|
@ -25,12 +25,11 @@
|
||||
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 RMCommunicator(ClientService clientService, AppContext context) {
|
||||
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 @@ protected AMRMProtocol createSchedulerProxy() {
|
||||
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 {
|
||||
|
@ -40,7 +40,6 @@
|
||||
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 void init(Configuration conf) {
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -31,7 +31,6 @@
|
||||
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 ContainerRequest(ContainerRequestEvent event, Priority priority) {
|
||||
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);
|
||||
|
@ -34,13 +34,13 @@
|
||||
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 DefaultSpeculator(Configuration conf, AppContext context, Clock clock) {
|
||||
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 DefaultSpeculator(Configuration conf, AppContext context, Clock clock) {
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -23,8 +23,8 @@
|
||||
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 @@ private EstimateVector getEstimateVector(TaskAttemptId attemptID) {
|
||||
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;
|
||||
}
|
||||
|
||||
|
@ -35,7 +35,6 @@
|
||||
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.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 MRApp(int maps, int reduces, boolean autoComplete, String testName, boole
|
||||
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*****");
|
||||
|
@ -131,7 +131,7 @@ public void testCrashed() throws Exception {
|
||||
//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);
|
||||
|
@ -25,9 +25,6 @@
|
||||
@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
|
||||
|
@ -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";
|
||||
}
|
@ -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;
|
||||
}
|
@ -38,8 +38,8 @@
|
||||
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 static PathFilter getHistoryFileFilter() {
|
||||
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 static String getConfiguredHistoryIntermediateDoneDirPrefix(
|
||||
*/
|
||||
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 static String getHistoryIntermediateDoneDirForUser(Configuration conf) th
|
||||
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 @@ private static Path[] filteredStat2Paths(List<FileStatus> stats, boolean dirs,
|
||||
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() ||
|
||||
|
@ -21,7 +21,7 @@
|
||||
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<? extends Annotation> annotationType() {
|
||||
|
||||
@Override
|
||||
public String serverPrincipal() {
|
||||
return JHConfig.HS_SERVER_PRINCIPAL_KEY;
|
||||
return JHAdminConfig.MR_HISTORY_PRINCIPAL;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -32,6 +32,7 @@
|
||||
|
||||
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 static void addToClassPath(
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -20,6 +20,7 @@
|
||||
|
||||
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.",
|
||||
|
@ -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";
|
||||
}
|
||||
|
@ -63,7 +63,7 @@
|
||||
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 void start() {
|
||||
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 void start() {
|
||||
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 void start() {
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -57,7 +57,7 @@
|
||||
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 void init(Configuration conf) throws YarnException {
|
||||
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 void init(Configuration conf) throws YarnException {
|
||||
|
||||
|
||||
|
||||
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 void start() {
|
||||
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),
|
||||
|
@ -24,7 +24,7 @@
|
||||
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 synchronized void init(Configuration conf) {
|
||||
}
|
||||
|
||||
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) {
|
||||
|
@ -27,7 +27,7 @@
|
||||
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 @@ synchronized ClientServiceDelegate getClient(JobID jobId) {
|
||||
|
||||
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)
|
||||
|
@ -26,7 +26,6 @@
|
||||
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.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 ResourceMgrDelegate(Configuration conf) {
|
||||
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(
|
||||
|
@ -59,7 +59,6 @@
|
||||
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 @@ private ApplicationSubmissionContext createApplicationSubmissionContext(
|
||||
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 @@ private ApplicationSubmissionContext createApplicationSubmissionContext(
|
||||
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>();
|
||||
|
@ -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";
|
||||
}
|
@ -21,7 +21,6 @@
|
||||
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.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 void testRedirect() throws Exception {
|
||||
|
||||
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();
|
||||
|
@ -29,11 +29,10 @@
|
||||
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 MiniMRYarnCluster(String testName) {
|
||||
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
|
||||
|
@ -68,9 +68,6 @@
|
||||
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 void testSleepJobWithSecurityOn() throws IOException,
|
||||
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());
|
||||
|
||||
|
@ -67,7 +67,7 @@
|
||||
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 @@ class Shuffle extends SimpleChannelUpstreamHandler {
|
||||
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) {
|
||||
|
@ -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 YarnConfiguration(Configuration conf) {
|
||||
}
|
||||
|
||||
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, "/");
|
||||
}
|
||||
|
@ -23,16 +23,11 @@
|
||||
|
||||
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 static RecordFactory getRecordFactory(Configuration conf) {
|
||||
//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);
|
||||
|
@ -25,6 +25,7 @@
|
||||
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 @@
|
||||
*/
|
||||
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 static RpcServerFactory getServerFactory(Configuration conf) {
|
||||
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 static RpcServerFactory getServerFactory(Configuration conf) {
|
||||
}
|
||||
|
||||
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);
|
||||
|
@ -23,16 +23,12 @@
|
||||
|
||||
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 static YarnRemoteExceptionFactory getYarnRemoteExceptionFactory(Configura
|
||||
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);
|
||||
|
@ -27,6 +27,7 @@
|
||||
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 @@
|
||||
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 Server getServer(Class protocol, Object instance,
|
||||
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();
|
||||
|
@ -44,7 +44,7 @@ public Class<? extends Annotation> annotationType() {
|
||||
|
||||
@Override
|
||||
public String serverPrincipal() {
|
||||
return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY;
|
||||
return YarnConfiguration.RM_PRINCIPAL;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -38,6 +38,7 @@
|
||||
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 void testHadoopProtoRPC() throws Exception {
|
||||
|
||||
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);
|
||||
|
@ -21,6 +21,7 @@
|
||||
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 void testFactoryProvider() {
|
||||
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 void testFactoryProvider() {
|
||||
}
|
||||
|
||||
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);
|
||||
|
@ -32,6 +32,7 @@
|
||||
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 NodeHealthCheckerService(Configuration conf) {
|
||||
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 @@ private synchronized void setLastReportedTime(long lastReportedTime) {
|
||||
*/
|
||||
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;
|
||||
}
|
||||
|
@ -43,12 +43,12 @@ public Class<? extends Annotation> annotationType() {
|
||||
|
||||
@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;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -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";
|
||||
}
|
@ -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.24.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>
|
||||
|
@ -30,6 +30,7 @@
|
||||
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 void tearDown() throws Exception {
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
|
@ -33,6 +33,7 @@
|
||||
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 int launchContainer(Container container,
|
||||
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 @@ private void createAppLogDirs(String appId)
|
||||
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 @@ private void createContainerLogDirs(String appId, String containerId)
|
||||
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);
|
||||
|
@ -28,21 +28,15 @@
|
||||
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 void delete(String user, Path subDir, Path... baseDirs) {
|
||||
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);
|
||||
|
@ -33,6 +33,7 @@
|
||||
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 @@ protected String getContainerExecutorExecutablePath(Configuration conf) {
|
||||
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
|
||||
|
@ -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;
|
||||
|
||||
}
|
@ -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.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 @@ protected WebServer createWebServer(Context nmContext,
|
||||
}
|
||||
|
||||
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 void init(Configuration conf) {
|
||||
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);
|
||||
|
@ -47,7 +47,6 @@
|
||||
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 NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
|
||||
@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 synchronized void init(Configuration conf) {
|
||||
@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 {
|
||||
|
@ -31,6 +31,7 @@
|
||||
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 Map<String, ByteBuffer> getMeta() {
|
||||
|
||||
@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);
|
||||
|
@ -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.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.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 @@ protected ContainersLauncher createContainersLauncher(Context context,
|
||||
@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 void start() {
|
||||
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();
|
||||
|
@ -45,10 +45,10 @@
|
||||
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 ContainerLaunch(Configuration configuration, Dispatcher dispatcher,
|
||||
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 Integer call() {
|
||||
|
||||
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 Integer call() {
|
||||
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);
|
||||
|
@ -50,18 +50,6 @@
|
||||
|
||||
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.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 ResourceLocalizationService(Dispatcher dispatcher,
|
||||
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 void init(Configuration conf) {
|
||||
// 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 void init(Configuration conf) {
|
||||
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 void init(Configuration conf) {
|
||||
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 @@ Server createServer() {
|
||||
|
||||
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 @@ class PublicLocalizer extends Thread {
|
||||
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>>());
|
||||
}
|
||||
|
@ -42,6 +42,7 @@
|
||||
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 FSDataOutputStream run() throws Exception {
|
||||
// 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 {
|
||||
|
@ -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.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 LogAggregationService(DeletionService deletionService) {
|
||||
|
||||
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 =
|
||||
|
@ -40,7 +40,6 @@
|
||||
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 int run(String[] args) throws Exception {
|
||||
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 @@ private int dumpAContainerLogs(String containerIdStr,
|
||||
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 =
|
||||
|
@ -29,11 +29,11 @@
|
||||
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 ContainersMonitorImpl(ContainerExecutor exec,
|
||||
@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 synchronized void init(Configuration conf) {
|
||||
|
||||
// ///////// 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 synchronized void init(Configuration conf) {
|
||||
|
||||
// ///////// 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
|
||||
|
@ -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.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 static class ContainersLogsBlock extends HtmlBlock implements
|
||||
@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 @@ protected void render(Block html) {
|
||||
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());
|
||||
|
@ -21,7 +21,6 @@
|
||||
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;
|
||||
|
@ -24,8 +24,8 @@
|
||||
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 synchronized void init(Configuration conf) {
|
||||
|
||||
@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 =
|
||||
|
@ -26,6 +26,7 @@
|
||||
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 void testLocalFilesCleanup() throws InterruptedException,
|
||||
|
||||
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);
|
||||
|
@ -74,9 +74,9 @@ public void testSuccessfulContainerLaunch() throws InterruptedException,
|
||||
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);
|
||||
|
@ -207,13 +207,13 @@ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
||||
};
|
||||
|
||||
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() {
|
||||
|
@ -22,6 +22,7 @@
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -24,6 +24,7 @@
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -47,7 +47,6 @@
|
||||
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 void setup() throws IOException {
|
||||
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) {
|
||||
|
@ -27,6 +27,7 @@
|
||||
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 @@ static class ServiceB extends LightService {
|
||||
@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 void testAuxEventDispatch() {
|
||||
@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 void 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 void testAuxServicesMeta() {
|
||||
@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);
|
||||
|
@ -46,6 +46,7 @@
|
||||
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 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 void testLocalizationInit() throws Exception {
|
||||
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 void testLocalizationHeartbeat() throws Exception {
|
||||
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();
|
||||
|
@ -47,11 +47,11 @@
|
||||
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 void tearDown() throws IOException, InterruptedException {
|
||||
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 void testLocalFileDeletionAfterUpload() throws IOException {
|
||||
|
||||
@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 void testNoContainerOnNode() {
|
||||
@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);
|
||||
|
@ -47,6 +47,7 @@
|
||||
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 TestContainersMonitor() throws UnsupportedFileSystemException {
|
||||
@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();
|
||||
}
|
||||
|
@ -28,12 +28,12 @@
|
||||
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 long getPmemAllocatedForContainers() {
|
||||
};
|
||||
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();
|
||||
|
||||
|
@ -43,7 +43,7 @@ public Class<? extends Annotation> annotationType() {
|
||||
|
||||
@Override
|
||||
public String serverPrincipal() {
|
||||
return YarnConfiguration.RM_SERVER_PRINCIPAL_KEY;
|
||||
return YarnConfiguration.RM_PRINCIPAL;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -83,12 +83,12 @@ public AdminService(Configuration conf, ResourceScheduler scheduler,
|
||||
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 void start() {
|
||||
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 RefreshAdminAclsResponse refreshAdminAcls(
|
||||
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");
|
||||
|
||||
|
@ -25,6 +25,7 @@
|
||||
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 ApplicationACLsManager(Configuration conf) {
|
||||
}
|
||||
|
||||
public boolean areACLsEnabled() {
|
||||
return conf.getBoolean(RMConfig.RM_ACLS_ENABLED, false);
|
||||
return conf.getBoolean(YarnConfiguration.RM_ACL_ENABLE,
|
||||
YarnConfiguration.DEFAULT_RM_ACL_ENABLE);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -90,8 +90,8 @@ public ApplicationMasterService(RMContext rmContext,
|
||||
@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 void start() {
|
||||
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();
|
||||
}
|
||||
|
@ -31,7 +31,6 @@
|
||||
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.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.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 ClientRMService(RMContext rmContext, YarnScheduler scheduler) {
|
||||
@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 void start() {
|
||||
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();
|
||||
}
|
||||
|
@ -21,6 +21,7 @@
|
||||
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 NMLivelinessMonitor(Dispatcher d) {
|
||||
|
||||
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
|
||||
|
@ -26,6 +26,7 @@
|
||||
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 void init(Configuration conf) {
|
||||
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 @@ private void printConfiguredHosts() {
|
||||
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);
|
||||
}
|
||||
|
@ -28,6 +28,7 @@
|
||||
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 RMAppManager(RMContext context, ClientToAMSecretManager
|
||||
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));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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;
|
||||
}
|
@ -202,7 +202,7 @@ protected void addIfService(Object object) {
|
||||
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 void handle(RMNodeEvent event) {
|
||||
|
||||
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 void start() {
|
||||
}
|
||||
|
||||
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
|
||||
|
@ -44,7 +44,6 @@
|
||||
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 ResourceTrackerService(RMContext rmContext,
|
||||
@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 synchronized void start() {
|
||||
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();
|
||||
|
||||
}
|
||||
|
@ -33,7 +33,6 @@
|
||||
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 @@ private Map<String, String> setupTokensInEnv(
|
||||
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 =
|
||||
|
@ -23,14 +23,14 @@
|
||||
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;
|
||||
|
@ -41,6 +41,7 @@
|
||||
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.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 void process(WatchedEvent arg0) {
|
||||
|
||||
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()
|
||||
|
@ -37,9 +37,9 @@
|
||||
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 RMAppImpl(ApplicationId applicationId, RMContext rmContext,
|
||||
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();
|
||||
|
@ -24,7 +24,6 @@
|
||||
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 AMLivelinessMonitor(Dispatcher d) {
|
||||
|
||||
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
|
||||
|
@ -21,9 +21,9 @@
|
||||
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 ContainerAllocationExpirer(Dispatcher d) {
|
||||
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
|
||||
|
@ -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 = ".";
|
||||
|
@ -51,7 +51,6 @@
|
||||
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.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";
|
||||
|
@ -35,7 +35,6 @@
|
||||
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 @@ private RMAdminProtocol createAdminProtocol() throws IOException {
|
||||
|
||||
// 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()) {
|
||||
|
@ -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>
|
||||
|
@ -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.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.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 TestRMAppManager extends RMAppManager {
|
||||
|
||||
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
Loading…
Reference in New Issue
Block a user