Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -615,12 +615,12 @@ public boolean isUntrackedNode(String hostName) {
}

/**
* Refresh the nodes gracefully
* Refresh the nodes gracefully.
*
* @param yarnConf
* @param yarnConf yarn configuration.
* @param timeout decommission timeout, null means default timeout.
* @throws IOException
* @throws YarnException
* @throws IOException io error occur.
* @throws YarnException exceptions from yarn servers.
*/
public void refreshNodesGracefully(Configuration yarnConf, Integer timeout)
throws IOException, YarnException {
Expand Down Expand Up @@ -682,6 +682,9 @@ private int readDecommissioningTimeout(Configuration pConf) {
/**
* A NodeId instance needed upon startup for populating inactive nodes Map.
* It only knows the hostname/ip and marks the port to -1 or invalid.
*
* @param host host name.
* @return node id.
*/
public static NodeId createUnknownNodeId(String host) {
return NodeId.newInstance(host, -1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -489,6 +489,7 @@ public static void logFailure(String user, String operation, String perm,
* @param description Some additional information as to why the operation
* failed.
* @param appId ApplicationId in which operation was performed.
* @param attemptId Application Attempt Id in which operation was performed.
*
* <br><br>
* Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,8 @@ public class RMSecretManagerService extends AbstractService {

/**
* Construct the service.
*
* @param conf Configuration.
* @param rmContext RMContext.
*/
public RMSecretManagerService(Configuration conf, RMContextImpl rmContext) {
super(RMSecretManagerService.class.getName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -241,7 +241,15 @@ private static String validateContainerIdAndVersion(

/**
* Utility method to validate a list resource requests, by ensuring that the
* requested memory/vcore is non-negative and not greater than max
* requested memory/vcore is non-negative and not greater than max.
*
* @param ask resource request.
* @param maximumAllocation Maximum Allocation.
* @param queueName queue name.
* @param scheduler YarnScheduler.
* @param rmContext RMContext.
* @param nodeLabelsEnabled the node labels feature enabled.
* @throws InvalidResourceRequestException when there is invalid request.
*/
public static void normalizeAndValidateRequests(List<ResourceRequest> ask,
Resource maximumAllocation, String queueName, YarnScheduler scheduler,
Expand All @@ -264,9 +272,13 @@ public static void normalizeAndValidateRequests(List<ResourceRequest> ask,

/**
* Validate increase/decrease request.
*
* <pre>
* - Throw exception when any other error happens
* </pre>
* @param request SchedContainerChangeRequest.
* @param increase true, add container; false, decrease container.
* @throws InvalidResourceRequestException when there is invalid request.
*/
public static void checkSchedContainerChangeRequest(
SchedContainerChangeRequest request, boolean increase)
Expand Down Expand Up @@ -362,6 +374,7 @@ private static boolean validateIncreaseDecreaseRequest(RMContext rmContext,
* application master.
* @param appAttemptId Application attempt Id
* @throws InvalidContainerReleaseException
* an Application Master tries to release containers not belonging to it using.
*/
public static void
validateContainerReleaseRequest(List<ContainerId> containerReleaseList,
Expand Down Expand Up @@ -394,7 +407,7 @@ public static UserGroupInformation verifyAdminAccess(
* @param module like AdminService or NodeLabelManager
* @param LOG the logger to use
* @return {@link UserGroupInformation} of the current user
* @throws IOException
* @throws IOException an I/O exception has occurred.
*/
public static UserGroupInformation verifyAdminAccess(
YarnAuthorizationProvider authorizer, String method, String module,
Expand Down Expand Up @@ -509,7 +522,9 @@ public static YarnApplicationAttemptState createApplicationAttemptState(
/**
* Find all configs whose name starts with
* YarnConfiguration.RM_PROXY_USER_PREFIX, and add a record for each one by
* replacing the prefix with ProxyUsers.CONF_HADOOP_PROXYUSER
* replacing the prefix with ProxyUsers.CONF_HADOOP_PROXYUSER.
*
* @param conf Configuration.
*/
public static void processRMProxyUsersConf(Configuration conf) {
Map<String, String> rmProxyUsers = new HashMap<String, String>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1589,6 +1589,7 @@ private boolean isOpportunisticSchedulingEnabled(Configuration conf) {

/**
* Create RMDelegatedNodeLabelsUpdater based on configuration.
* @return RMDelegatedNodeLabelsUpdater.
*/
protected RMDelegatedNodeLabelsUpdater createRMDelegatedNodeLabelsUpdater() {
if (conf.getBoolean(YarnConfiguration.NODE_LABELS_ENABLED,
Expand Down Expand Up @@ -1745,9 +1746,9 @@ private void setSchedulerRecoveryStartAndWaitTime(RMState state,
}

/**
* Retrieve RM bind address from configuration
* Retrieve RM bind address from configuration.
*
* @param conf
* @param conf Configuration.
* @return InetSocketAddress
*/
public static InetSocketAddress getBindAddress(Configuration conf) {
Expand All @@ -1758,8 +1759,8 @@ public static InetSocketAddress getBindAddress(Configuration conf) {
/**
* Deletes the RMStateStore
*
* @param conf
* @throws Exception
* @param conf Configuration.
* @throws Exception error occur.
*/
@VisibleForTesting
static void deleteRMStateStore(Configuration conf) throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -194,8 +194,8 @@ protected void serviceInit(Configuration conf) throws Exception {

/**
* Load DynamicResourceConfiguration from dynamic-resources.xml.
* @param conf
* @throws IOException
* @param conf Configuration.
* @throws IOException an I/O exception has occurred.
*/
public void loadDynamicResourceConfiguration(Configuration conf)
throws IOException {
Expand All @@ -219,7 +219,7 @@ public void loadDynamicResourceConfiguration(Configuration conf)

/**
* Update DynamicResourceConfiguration with new configuration.
* @param conf
* @param conf DynamicResourceConfiguration.
*/
public void updateDynamicResourceConfiguration(
DynamicResourceConfiguration conf) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,7 @@ private static void deductPreemptableResourcePerApp(
* stop preempt container when: all major resource type
* {@literal <=} 0 for to-preempt.
* This is default preemption behavior of inter-queue preemption
* @param curCandidates RMContainer Set.
* @return should we preempt rmContainer. If we should, deduct from
* <code>resourceToObtainByPartition</code>
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,13 +42,22 @@ public class PreemptableResourceCalculator
LoggerFactory.getLogger(PreemptableResourceCalculator.class);

/**
* PreemptableResourceCalculator constructor
* PreemptableResourceCalculator constructor.
*
* @param preemptionContext
* @param preemptionContext context.
* @param isReservedPreemptionCandidatesSelector this will be set by
* different implementation of candidate selectors, please refer to
* TempQueuePerPartition#offer for details.
* @param allowQueuesBalanceAfterAllQueuesSatisfied
* Should resources be preempted from an over-served queue when the
* requesting queues are all at or over their guarantees?
* An example is, there're 10 queues under root, guaranteed resource
* of them are all 10%.
* Assume there're two queues are using resources, queueA uses 10%
* queueB uses 90%. For all queues are guaranteed, but it's not fair
* for queueA.
* We wanna make this behavior can be configured. By default it is
* not allowed.
*/
public PreemptableResourceCalculator(
CapacitySchedulerPreemptionContext preemptionContext,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -272,9 +272,11 @@ private void replaceNodeToAttribute(String nodeHost, String prefix,
}

/**
* @param nodeAttributeMapping
* @param newAttributesToBeAdded
* @param isRemoveOperation : to indicate whether its a remove operation.
* Validate for attributes.
*
* @param nodeAttributeMapping NodeAttribute Mapping
* @param newAttributesToBeAdded new Attributes ToBeAdded
* @param isRemoveOperation : to indicate whether it's a remove operation.
* @return Map of String to Map of NodeAttribute to AttributeValue
* @throws IOException : on invalid mapping in the current request or against
* already existing NodeAttributes.
Expand Down Expand Up @@ -329,12 +331,13 @@ protected Map<String, Map<NodeAttribute, AttributeValue>> validate(
}

/**
* Validate For AttributeType Mismatch.
*
* @param isRemoveOperation
* @param attribute
* @param newAttributes
* @return Whether its a new Attribute added
* @throws IOException
* @param isRemoveOperation to indicate whether it's a remove operation.
* @param attribute NodeAttribute.
* @param newAttributes new Attributes.
* @return Whether it's a new Attribute added
* @throws IOException an I/O exception of some sort has occurred.
*/
private boolean validateForAttributeTypeMismatch(boolean isRemoveOperation,
NodeAttribute attribute,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,8 @@ protected void serviceStart() throws Exception {

/**
* Terminate the timer.
* @throws Exception
*
* @throws Exception exception occurs.
*/
@Override
protected void serviceStop() throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@ private PlacementFactory() {
* @param ruleStr The name of the class to instantiate
* @param conf The configuration object to set for the rule
* @return Created class instance
* @throws ClassNotFoundException
* no definition for the class with the specified name could be found.
*/
public static PlacementRule getPlacementRule(String ruleStr,
Configuration conf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ public abstract class PlacementRule {
/**
* Set the config based on the passed in argument. This construct is used to
* not pollute this abstract class with implementation specific references.
* @param initArg initialization arguments.
*/
public void setConfig(Object initArg) {
// Default is a noop
Expand Down
Loading