diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java index f9ed776..ae54bc9 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java @@ -20,7 +20,6 @@ import java.io.IOException; -import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; @@ -73,7 +72,7 @@ * a new application.

* *

The ResourceManager also responds with details such - * as minimum and maximum resource capabilities in the cluster as specified in + * as maximum resource capabilities in the cluster as specified in * {@link GetNewApplicationResponse}.

* * @param request request to get a new ApplicationId @@ -83,6 +82,8 @@ * @throws IOException * @see #submitApplication(SubmitApplicationRequest) */ + @Public + @Stable public GetNewApplicationResponse getNewApplication( GetNewApplicationRequest request) throws YarnException, IOException; @@ -112,6 +113,8 @@ public GetNewApplicationResponse getNewApplication( * @throws IOException * @see #getNewApplication(GetNewApplicationRequest) */ + @Public + @Stable public SubmitApplicationResponse submitApplication( SubmitApplicationRequest request) throws YarnException, IOException; @@ -129,13 +132,15 @@ public SubmitApplicationResponse submitApplication( *

Currently, the ResourceManager returns an empty response * on success and throws an exception on rejecting the request.

* - * @param request request to abort a submited application + * @param request request to abort a submitted application * @return ResourceManager returns an empty response * on success and throws an exception on rejecting the request * @throws YarnException * @throws IOException * @see #getQueueUserAcls(GetQueueUserAclsInfoRequest) */ + @Public + @Stable public KillApplicationResponse forceKillApplication( KillApplicationRequest request) throws YarnException, IOException; @@ -171,6 +176,8 @@ public KillApplicationResponse forceKillApplication( * @throws YarnException * @throws IOException */ + @Public + @Stable public GetApplicationReportResponse getApplicationReport( GetApplicationReportRequest request) throws YarnException, IOException; @@ -189,6 +196,8 @@ public GetApplicationReportResponse getApplicationReport( * @throws YarnException * @throws IOException */ + @Public + @Stable public GetClusterMetricsResponse getClusterMetrics( GetClusterMetricsRequest request) throws YarnException, IOException; @@ -211,6 +220,8 @@ public GetClusterMetricsResponse getClusterMetrics( * @throws YarnException * @throws IOException */ + @Public + @Stable public GetAllApplicationsResponse getAllApplications( GetAllApplicationsRequest request) throws YarnException, IOException; @@ -228,6 +239,8 @@ public GetAllApplicationsResponse getAllApplications( * @throws YarnException * @throws IOException */ + @Public + @Stable public GetClusterNodesResponse getClusterNodes( GetClusterNodesRequest request) throws YarnException, IOException; @@ -247,6 +260,8 @@ public GetClusterNodesResponse getClusterNodes( * @throws YarnException * @throws IOException */ + @Public + @Stable public GetQueueInfoResponse getQueueInfo( GetQueueInfoRequest request) throws YarnException, IOException; @@ -264,6 +279,8 @@ public GetQueueInfoResponse getQueueInfo( * @throws YarnException * @throws IOException */ + @Public + @Stable public GetQueueUserAclsInfoResponse getQueueUserAcls( GetQueueUserAclsInfoRequest request) throws YarnException, IOException; @@ -272,7 +289,7 @@ public GetQueueUserAclsInfoResponse getQueueUserAcls( *

The interface used by clients to get delegation token, enabling the * containers to be able to talk to the service using those tokens. * - *

The ResourceManager responds with the delegation token + *

The ResourceManager responds with the delegation * {@link Token} that can be used by the client to speak to this * service. * @param request request to get a delegation token for the client. @@ -280,32 +297,36 @@ public GetQueueUserAclsInfoResponse getQueueUserAcls( * @throws YarnException * @throws IOException */ + @Public + @Stable public GetDelegationTokenResponse getDelegationToken( GetDelegationTokenRequest request) throws YarnException, IOException; /** - * Renew an existing delegation token. + * Renew an existing delegation {@link Token}. * * @param request the delegation token to be renewed. * @return the new expiry time for the delegation token. * @throws YarnException * @throws IOException */ - @Private + @Public + @Stable public RenewDelegationTokenResponse renewDelegationToken( RenewDelegationTokenRequest request) throws YarnException, IOException; /** - * Cancel an existing delegation token. + * Cancel an existing delegation {@link Token}. * * @param request the delegation token to be cancelled. * @return an empty response. * @throws YarnException * @throws IOException */ - @Private + @Public + @Stable public CancelDelegationTokenResponse cancelDelegationToken( CancelDelegationTokenRequest request) throws YarnException, IOException; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocolPB.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocolPB.java index 7f8cd38..58d71aa 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocolPB.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocolPB.java @@ -17,9 +17,11 @@ */ package org.apache.hadoop.yarn.api; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.yarn.proto.ApplicationClientProtocol.ApplicationClientProtocolService; +@Private @ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ApplicationClientProtocolPB", protocolVersion = 1) public interface ApplicationClientProtocolPB extends ApplicationClientProtocolService.BlockingInterface { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java index 9600142..39a72ed 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java @@ -18,6 +18,8 @@ package org.apache.hadoop.yarn.api; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; @@ -27,9 +29,15 @@ * * TODO: Investigate the semantics and security of each cross-boundary refs. */ +@Public +@Evolving public interface ApplicationConstants { - // TODO: They say tokens via env isn't good. + /** + * The application client token secret key sent to AppMaster via environment + * on AppMaster launch, used to verify corresponding application client token + * TODO: They say tokens via env isn't good. + */ public static final String APPLICATION_CLIENT_SECRET_ENV_NAME = "AppClientSecretEnv"; @@ -39,6 +47,9 @@ */ public static final String APP_SUBMIT_TIME_ENV = "APP_SUBMIT_TIME_ENV"; + /** + * The cache file into which container token is written + */ public static final String CONTAINER_TOKEN_FILE_ENV_NAME = UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION; @@ -50,6 +61,10 @@ public static final String APPLICATION_WEB_PROXY_BASE_ENV = "APPLICATION_WEB_PROXY_BASE"; + /** + * The temporary environmental variable for container log directory. This + * should be replaced by real container log directory on container launch. + */ public static final String LOG_DIR_EXPANSION_VAR = ""; public static final String STDERR = "stderr"; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocol.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocol.java index 4baa874..bdf92fb 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocol.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocol.java @@ -54,7 +54,7 @@ * {@link RegisterApplicationMasterRequest}.

* *

The ResourceManager responds with critical details such - * as minimum and maximum resource capabilities in the cluster as specified in + * as maximum resource capabilities in the cluster as specified in * {@link RegisterApplicationMasterResponse}.

* * @param request registration request @@ -64,6 +64,8 @@ * @see RegisterApplicationMasterRequest * @see RegisterApplicationMasterResponse */ + @Public + @Stable public RegisterApplicationMasterResponse registerApplicationMaster( RegisterApplicationMasterRequest request) throws YarnException, IOException; @@ -86,6 +88,8 @@ public RegisterApplicationMasterResponse registerApplicationMaster( * @see FinishApplicationMasterRequest * @see FinishApplicationMasterResponse */ + @Public + @Stable public FinishApplicationMasterResponse finishApplicationMaster( FinishApplicationMasterRequest request) throws YarnException, IOException; @@ -122,6 +126,8 @@ public FinishApplicationMasterResponse finishApplicationMaster( * @see AllocateRequest * @see AllocateResponse */ + @Public + @Stable public AllocateResponse allocate(AllocateRequest request) throws YarnException, IOException; } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocolPB.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocolPB.java index 027a6f0..17a62ab 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocolPB.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationMasterProtocolPB.java @@ -17,9 +17,11 @@ */ package org.apache.hadoop.yarn.api; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService; +@Private @ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB", protocolVersion = 1) public interface ApplicationMasterProtocolPB extends ApplicationMasterProtocolService.BlockingInterface { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java index 86a9edc..ca40b44 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocolPB.java @@ -17,9 +17,11 @@ */ package org.apache.hadoop.yarn.api; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.yarn.proto.ContainerManagementProtocol.ContainerManagementProtocolService; +@Private @ProtocolInfo( protocolName = "org.apache.hadoop.yarn.api.ContainerManagementProtocolPB", protocolVersion = 1) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocol.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocol.java index 76916b0..ebc1c43 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocol.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocol.java @@ -20,6 +20,8 @@ import java.io.IOException; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.tools.GetUserMappingsProtocol; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.api.protocolrecords.RefreshAdminAclsRequest; @@ -35,26 +37,41 @@ import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsRequest; import org.apache.hadoop.yarn.api.protocolrecords.RefreshUserToGroupsMappingsResponse; +@Public +@Stable public interface ResourceManagerAdministrationProtocol extends GetUserMappingsProtocol { + + @Public + @Stable public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request) throws YarnException, IOException; - + + @Public + @Stable public RefreshNodesResponse refreshNodes(RefreshNodesRequest request) throws YarnException, IOException; - + + @Public + @Stable public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration( RefreshSuperUserGroupsConfigurationRequest request) throws YarnException, IOException; + @Public + @Stable public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings( RefreshUserToGroupsMappingsRequest request) throws YarnException, IOException; - + + @Public + @Stable public RefreshAdminAclsResponse refreshAdminAcls( RefreshAdminAclsRequest request) throws YarnException, IOException; - + + @Public + @Stable public RefreshServiceAclsResponse refreshServiceAcls( RefreshServiceAclsRequest request) throws YarnException, IOException; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocolPB.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocolPB.java index 5042726..e353ec0 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocolPB.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ResourceManagerAdministrationProtocolPB.java @@ -17,9 +17,11 @@ */ package org.apache.hadoop.yarn.api; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.ipc.ProtocolInfo; import org.apache.hadoop.yarn.proto.ResourceManagerAdministrationProtocol.ResourceManagerAdministrationProtocolService; +@Private @ProtocolInfo( protocolName = "org.apache.hadoop.yarn.api.ResourceManagerAdministrationProtocolPB", protocolVersion = 1) diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/package-info.java new file mode 100644 index 0000000..d1f4ea7 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ +@InterfaceAudience.Public +package org.apache.hadoop.yarn.api; +import org.apache.hadoop.classification.InterfaceAudience; + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java index 15ab399..964d368 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java @@ -59,6 +59,8 @@ @Stable public abstract class AllocateRequest { + @Public + @Stable public static AllocateRequest newInstance( ApplicationAttemptId applicationAttemptId, int responseID, float appProgress, List resourceAsk, diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java index 689368f..3de74f7 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java @@ -43,10 +43,10 @@ * *

* @@ -51,7 +51,7 @@ public abstract class ApplicationReport { @Private - @Stable + @Unstable public static ApplicationReport newInstance(ApplicationId applicationId, ApplicationAttemptId applicationAttemptId, String user, String queue, String name, String host, int rpcPort, Token clientToken, @@ -98,7 +98,7 @@ public static ApplicationReport newInstance(ApplicationId applicationId, * attempt of the application * @return ApplicationAttemptId of the attempt */ - @Private + @Public @Unstable public abstract ApplicationAttemptId getCurrentApplicationAttemptId(); @@ -235,7 +235,7 @@ public static ApplicationReport newInstance(ApplicationId applicationId, * This is intended to only be used by the proxy itself. * @return the original not-proxied tracking url for the application */ - @Private + @Public @Unstable public abstract String getOriginalTrackingUrl(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java index 4cf2b14..db49a34 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java @@ -32,6 +32,7 @@ public abstract class ApplicationResourceUsageReport { @Private + @Unstable public static ApplicationResourceUsageReport newInstance( int numUsedContainers, int numReservedContainers, Resource usedResources, Resource reservedResources, Resource neededResources) { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java index 3d669b0..63adf03 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java @@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.util.Records; @@ -183,21 +182,21 @@ public static ApplicationSubmissionContext newInstance( * @return true if the AM is not managed by the RM */ @Public - @Unstable + @Stable public abstract boolean getUnmanagedAM(); /** * @param value true if RM should not manage the AM */ @Public - @Unstable + @Stable public abstract void setUnmanagedAM(boolean value); /** * @return true if tokens should be canceled when the app completes. */ @LimitedPrivate("mapreduce") - @Unstable + @Stable public abstract boolean getCancelTokensWhenComplete(); /** @@ -207,14 +206,14 @@ public static ApplicationSubmissionContext newInstance( * @param cancel true if tokens should be canceled when the app finishes. */ @LimitedPrivate("mapreduce") - @Unstable + @Stable public abstract void setCancelTokensWhenComplete(boolean cancel); /** * @return the number of max attempts of the application to be submitted */ @Public - @Unstable + @Stable public abstract int getMaxAppAttempts(); /** @@ -225,7 +224,7 @@ public static ApplicationSubmissionContext newInstance( * to be submitted. */ @Public - @Unstable + @Stable public abstract void setMaxAppAttempts(int maxAppAttempts); @Public @@ -237,19 +236,18 @@ public static ApplicationSubmissionContext newInstance( public abstract void setResource(Resource resource); /** - * Get the applicationType is the application type + * Get the application type * - * @return applicationType is the application type + * @return the application type */ @Public @Stable public abstract String getApplicationType(); /** - * Set the applicationType is the application type + * Set the application type * - * @param applicationType - * applicationType is the application type + * @param applicationType the application type */ @Public @Stable diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java index fb205f8..cb8d04b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java @@ -46,7 +46,7 @@ *
  • {@link Priority} at which the container was allocated.
  • *
  • {@link ContainerState} of the container.
  • *
  • - * Container Token {@link Token} of the container, used to securely verify + * Container {@link Token} of the container, used to securely verify * authenticity of the allocation. *
  • *
  • {@link ContainerStatus} of the container.
  • @@ -55,7 +55,7 @@ * *

    Typically, an ApplicationMaster receives the * Container from the ResourceManager during - * resource-negotiation and then talks to the NodManager to + * resource-negotiation and then talks to the NodeManager to * start/stop containers.

    * * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) @@ -67,6 +67,7 @@ public abstract class Container implements Comparable { @Private + @Unstable public static Container newInstance(ContainerId containerId, NodeId nodeId, String nodeHttpAddress, Resource resource, Priority priority, Token containerToken) { @@ -134,6 +135,8 @@ public static Container newInstance(ContainerId containerId, NodeId nodeId, * @return Priority at which the Container was * allocated */ + @Public + @Stable public abstract Priority getPriority(); @Private diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java index 95a989a..ccbe6f1 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java @@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.util.Records; /** @@ -34,6 +35,7 @@ public abstract class ContainerId implements Comparable{ @Private + @Unstable public static ContainerId newInstance(ApplicationAttemptId appAttemptId, int containerId) { ContainerId id = Records.newRecord(ContainerId.class); @@ -54,6 +56,7 @@ public static ContainerId newInstance(ApplicationAttemptId appAttemptId, public abstract ApplicationAttemptId getApplicationAttemptId(); @Private + @Unstable protected abstract void setApplicationAttemptId(ApplicationAttemptId atId); /** @@ -65,6 +68,7 @@ public static ContainerId newInstance(ApplicationAttemptId appAttemptId, public abstract int getId(); @Private + @Unstable protected abstract void setId(int id); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java index b7ff537..323d31d 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerState.java @@ -18,9 +18,14 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; + /** *

    State of a Container.

    */ +@Public +@Stable public enum ContainerState { /** New container */ NEW, diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java index 6322ee1..dcb1626 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java @@ -42,6 +42,7 @@ public abstract class ContainerStatus { @Private + @Unstable public static ContainerStatus newInstance(ContainerId containerId, ContainerState containerState, String diagnostics, int exitStatus) { ContainerStatus containerStatus = Records.newRecord(ContainerStatus.class); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java index f2a3eff..a220e6e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java @@ -69,24 +69,32 @@ public static LocalResource newInstance(URL url, LocalResourceType type, * Get the location of the resource to be localized. * @return location of the resource to be localized */ + @Public + @Stable public abstract URL getResource(); /** * Set location of the resource to be localized. * @param resource location of the resource to be localized */ + @Public + @Stable public abstract void setResource(URL resource); /** * Get the size of the resource to be localized. * @return size of the resource to be localized */ + @Public + @Stable public abstract long getSize(); /** * Set the size of the resource to be localized. * @param size size of the resource to be localized */ + @Public + @Stable public abstract void setSize(long size); /** @@ -94,6 +102,8 @@ public static LocalResource newInstance(URL url, LocalResourceType type, * for verification. * @return timestamp of the resource to be localized */ + @Public + @Stable public abstract long getTimestamp(); /** @@ -101,18 +111,24 @@ public static LocalResource newInstance(URL url, LocalResourceType type, * for verification. * @param timestamp timestamp of the resource to be localized */ + @Public + @Stable public abstract void setTimestamp(long timestamp); /** * Get the LocalResourceType of the resource to be localized. * @return LocalResourceType of the resource to be localized */ + @Public + @Stable public abstract LocalResourceType getType(); /** * Set the LocalResourceType of the resource to be localized. * @param type LocalResourceType of the resource to be localized */ + @Public + @Stable public abstract void setType(LocalResourceType type); /** @@ -121,6 +137,8 @@ public static LocalResource newInstance(URL url, LocalResourceType type, * @return LocalResourceVisibility of the resource to be * localized */ + @Public + @Stable public abstract LocalResourceVisibility getVisibility(); /** @@ -129,6 +147,8 @@ public static LocalResource newInstance(URL url, LocalResourceType type, * @param visibility LocalResourceVisibility of the resource to be * localized */ + @Public + @Stable public abstract void setVisibility(LocalResourceVisibility visibility); /** @@ -137,6 +157,8 @@ public static LocalResource newInstance(URL url, LocalResourceType type, * @return pattern that should be used to extract entries from the * archive. */ + @Public + @Stable public abstract String getPattern(); /** @@ -145,5 +167,7 @@ public static LocalResource newInstance(URL url, LocalResourceType type, * @param pattern pattern that should be used to extract entries * from the archive. */ + @Public + @Stable public abstract void setPattern(String pattern); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java index 3009971..c1c5eb4 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java @@ -35,6 +35,8 @@ * {@link #ARCHIVE} - Archive, which is automatically unarchived by the * NodeManager. * + *
  • + * {@link #PATTERN} - A hybrid between {@link #ARCHIVE} and {@link #FILE}. * *

    * diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NMToken.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NMToken.java index a901447..a439993 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NMToken.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NMToken.java @@ -21,13 +21,35 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.util.Records; /** - * NMToken is returned by RM on AllocateResponse. + *

    The NMToken is used for authenticating communication with + * NodeManager

    + *

    It is issued by ResourceMananger when ApplicationMaster + * negotiates resource with ResourceManager and + * validated on NodeManager side.

    + * @see AllocateResponse#getNMTokens() */ +@Public +@Stable public abstract class NMToken { + @Private + public static NMToken newInstance(NodeId nodeId, Token token) { + NMToken nmToken = Records.newRecord(NMToken.class); + nmToken.setNodeId(nodeId); + nmToken.setToken(token); + return nmToken; + } + + /** + * Get the {@link NodeId} of the NodeManager for which the NMToken + * is used to authenticate. + * @return the {@link NodeId} of the NodeManager for which the + * NMToken is used to authenticate. + */ @Public @Stable public abstract NodeId getNodeId(); @@ -35,7 +57,11 @@ @Public @Stable public abstract void setNodeId(NodeId nodeId); - + + /** + * Get the {@link Token} used for authenticating with NodeManager + * @return the {@link Token} used for authenticating with NodeManager + */ @Public @Stable public abstract Token getToken(); @@ -43,12 +69,5 @@ @Public @Stable public abstract void setToken(Token token); - - @Private - public static NMToken newInstance(NodeId nodeId, Token token) { - NMToken nmToken = Records.newRecord(NMToken.class); - nmToken.setNodeId(nodeId); - nmToken.setToken(token); - return nmToken; - } + } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java index 3c22c58..c3f8595 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java @@ -21,6 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.util.Records; /** @@ -35,6 +36,7 @@ public abstract class NodeId implements Comparable { @Private + @Unstable public static NodeId newInstance(String host, int port) { NodeId nodeId = Records.newRecord(NodeId.class); nodeId.setHost(host); @@ -52,6 +54,7 @@ public static NodeId newInstance(String host, int port) { public abstract String getHost(); @Private + @Unstable protected abstract void setHost(String host); /** @@ -63,6 +66,7 @@ public static NodeId newInstance(String host, int port) { public abstract int getPort(); @Private + @Unstable protected abstract void setPort(int port); @Override diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java index 496d330..11742f5 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeReport.java @@ -47,6 +47,7 @@ public abstract class NodeReport { @Private + @Unstable public static NodeReport newInstance(NodeId nodeId, NodeState nodeState, String httpAddress, String rackName, Resource used, Resource capability, int numContainers, String healthReport, long lastHealthReportTime) { @@ -67,6 +68,8 @@ public static NodeReport newInstance(NodeId nodeId, NodeState nodeState, * Get the NodeId of the node. * @return NodeId of the node */ + @Public + @Stable public abstract NodeId getNodeId(); @Private @@ -77,6 +80,8 @@ public static NodeReport newInstance(NodeId nodeId, NodeState nodeState, * Get the NodeState of the node. * @return NodeState of the node */ + @Public + @Stable public abstract NodeState getNodeState(); @Private diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeState.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeState.java index 7345c25..14b211a 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeState.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeState.java @@ -18,9 +18,14 @@ package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; + /** *

    State of a Node.

    */ +@Public +@Stable public enum NodeState { /** New node */ NEW, diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java index 3535eeb..4dd3178 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContainer.java @@ -28,8 +28,12 @@ * @see PreemptionContract * @see StrictPreemptionContract */ +@Public +@Evolving public abstract class PreemptionContainer { + @Private + @Unstable public static PreemptionContainer newInstance(ContainerId id) { PreemptionContainer container = Records.newRecord(PreemptionContainer.class); container.setId(id); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java index c794aff..dc954ef 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionContract.java @@ -34,8 +34,12 @@ * the platform. * @see PreemptionMessage */ +@Public +@Evolving public abstract class PreemptionContract { + @Private + @Unstable public static PreemptionContract newInstance( List req, Set containers) { PreemptionContract contract = Records.newRecord(PreemptionContract.class); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java index 72bcada..976d181 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionMessage.java @@ -24,41 +24,43 @@ import org.apache.hadoop.yarn.util.Records; /** - * A {@link PreemptionMessage} is part of the RM-AM protocol, and it is used by + *

    A {@link PreemptionMessage} is part of the RM-AM protocol, and it is used by * the RM to specify resources that the RM wants to reclaim from this * ApplicationMaster (AM). The AM receives a {@link * StrictPreemptionContract} message encoding which containers the platform may * forcibly kill, granting it an opportunity to checkpoint state or adjust its * execution plan. The message may also include a {@link PreemptionContract} * granting the AM more latitude in selecting which resources to return to the - * cluster. + * cluster.

    * - * The AM should decode both parts of the message. The {@link + *

    The AM should decode both parts of the message. The {@link * StrictPreemptionContract} specifies particular allocations that the RM * requires back. The AM can checkpoint containers' state, adjust its execution * plan to move the computation, or take no action and hope that conditions that - * caused the RM to ask for the container will change. + * caused the RM to ask for the container will change.

    * - * In contrast, the {@link PreemptionContract} also includes a description of + *

    In contrast, the {@link PreemptionContract} also includes a description of * resources with a set of containers. If the AM releases containers matching * that profile, then the containers enumerated in {@link - * PreemptionContract#getContainers()} may not be killed. + * PreemptionContract#getContainers()} may not be killed.

    * - * Each preemption message reflects the RM's current understanding of the + *

    Each preemption message reflects the RM's current understanding of the * cluster state, so a request to return N containers may not * reflect containers the AM is releasing, recently exited containers the RM has * yet to learn about, or new containers allocated before the message was * generated. Conversely, an RM may request a different profile of containers in - * subsequent requests. + * subsequent requests.

    * - * The policy enforced by the RM is part of the scheduler. Generally, only + *

    The policy enforced by the RM is part of the scheduler. Generally, only * containers that have been requested consistently should be killed, but the - * details are not specified. + * details are not specified.

    */ @Public @Evolving public abstract class PreemptionMessage { + @Private + @Unstable public static PreemptionMessage newInstance(StrictPreemptionContract set, PreemptionContract contract) { PreemptionMessage message = Records.newRecord(PreemptionMessage.class); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionResourceRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionResourceRequest.java index d6982c7..ed5bc55 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionResourceRequest.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/PreemptionResourceRequest.java @@ -29,8 +29,12 @@ * @see PreemptionContract * @see AllocateRequest#setAskList(java.util.List) */ +@Public +@Evolving public abstract class PreemptionResourceRequest { + @Private + @Unstable public static PreemptionResourceRequest newInstance(ResourceRequest req) { PreemptionResourceRequest request = Records.newRecord(PreemptionResourceRequest.class); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java index e918056..0d825bf 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Priority.java @@ -27,6 +27,8 @@ * allocation * */ +@Public +@Stable public abstract class Priority implements Comparable { @Public @@ -41,12 +43,16 @@ public static Priority newInstance(int p) { * Get the assigned priority * @return the assigned priority */ + @Public + @Stable public abstract int getPriority(); /** * Set the assigned priority * @param priority the assigned priority */ + @Public + @Stable public abstract void setPriority(int priority); @Override diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java index ccd9191..c6777db 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueACL.java @@ -28,7 +28,7 @@ *

    * *

    - * The ACLs are one of: + * The ACL is one of: *

      *
    • {@link #SUBMIT_APPLICATIONS} - ACL to submit applications to the * queue.
    • diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java index 3a58bab..7146db2 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueInfo.java @@ -50,6 +50,7 @@ public abstract class QueueInfo { @Private + @Unstable public static QueueInfo newInstance(String queueName, float capacity, float maximumCapacity, float currentCapacity, List childQueues, List applications, diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java index eb2cbaa..01698de 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueState.java @@ -25,7 +25,7 @@ /** *

      State of a Queue.

      * - *

      A queue is one of: + *

      A queue is in one of: *

        *
      • {@link #RUNNING} - normal state.
      • *
      • {@link #STOPPED} - not accepting new application submissions. diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java index 21d75fc..022d457 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/QueueUserACLInfo.java @@ -39,6 +39,7 @@ public abstract class QueueUserACLInfo { @Private + @Unstable public static QueueUserACLInfo newInstance(String queueName, List acls) { QueueUserACLInfo info = Records.newRecord(QueueUserACLInfo.class); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java index 130d361..2200ad8 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceBlacklistRequest.java @@ -35,7 +35,9 @@ @Public @Stable public abstract class ResourceBlacklistRequest { - + + @Public + @Stable public static ResourceBlacklistRequest newInstance( List additions, List removals) { ResourceBlacklistRequest blacklistRequest = @@ -52,6 +54,8 @@ public static ResourceBlacklistRequest newInstance( * @return list of resources which should be added to the * application blacklist */ + @Public + @Stable public abstract List getBlacklistAdditions(); /** @@ -60,6 +64,8 @@ public static ResourceBlacklistRequest newInstance( * @param resourceNames list of resources which should be added to the * application blacklist */ + @Public + @Stable public abstract void setBlacklistAdditions(List resourceNames); /** @@ -69,6 +75,8 @@ public static ResourceBlacklistRequest newInstance( * @return list of resources which should be removed from the * application blacklist */ + @Public + @Stable public abstract List getBlacklistRemovals(); /** @@ -78,6 +86,8 @@ public static ResourceBlacklistRequest newInstance( * @param resourceNames list of resources which should be removed from the * application blacklist */ + @Public + @Stable public abstract void setBlacklistRemovals(List resourceNames); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java index f9e71eb..78bb38c 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java @@ -71,8 +71,13 @@ public static ResourceRequest newInstance(Priority priority, return request; } + @Public + @Stable public static class ResourceRequestComparator implements java.util.Comparator, Serializable { + + private static final long serialVersionUID = 1L; + @Override public int compare(ResourceRequest r1, ResourceRequest r2) { @@ -105,6 +110,8 @@ public int compare(ResourceRequest r1, ResourceRequest r2) { * @return whether the given host/rack string represents an arbitrary * host name */ + @Public + @Stable public static boolean isAnyLocation(String hostName) { return ANY.equals(hostName); } @@ -198,24 +205,24 @@ public static boolean isAnyLocation(String hostName) { public abstract boolean getRelaxLocality(); /** - * For a request at a network hierarchy level, set whether locality can be relaxed - * to that level and beyond. + *

        For a request at a network hierarchy level, set whether locality can be relaxed + * to that level and beyond.

        * - * If the flag is off on a rack-level ResourceRequest, + *

        If the flag is off on a rack-level ResourceRequest, * containers at that request's priority will not be assigned to nodes on that * request's rack unless requests specifically for those nodes have also been - * submitted. + * submitted.

        * - * If the flag is off on an {@link ResourceRequest#ANY}-level + *

        If the flag is off on an {@link ResourceRequest#ANY}-level * ResourceRequest, containers at that request's priority will * only be assigned on racks for which specific requests have also been - * submitted. + * submitted.

        * - * For example, to request a container strictly on a specific node, the + *

        For example, to request a container strictly on a specific node, the * corresponding rack-level and any-level requests should have locality * relaxation set to false. Similarly, to request a container strictly on a * specific rack, the corresponding any-level request should have locality - * relaxation set to false. + * relaxation set to false.

        * * @param relaxLocality whether locality relaxation is enabled with this * ResourceRequest. diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java index f702222..18ba25d 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/StrictPreemptionContract.java @@ -37,6 +37,8 @@ @Evolving public abstract class StrictPreemptionContract { + @Private + @Unstable public static StrictPreemptionContract newInstance(Set containers) { StrictPreemptionContract contract = Records.newRecord(StrictPreemptionContract.class); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Token.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Token.java index 8dfa376..ae2983b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Token.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Token.java @@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Stable; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.util.Records; /** @@ -34,6 +35,7 @@ public abstract class Token { @Private + @Unstable public static Token newInstance(byte[] identifier, String kind, byte[] password, String service) { Token token = Records.newRecord(Token.class); @@ -53,7 +55,7 @@ public static Token newInstance(byte[] identifier, String kind, byte[] password, public abstract ByteBuffer getIdentifier(); @Private - @Stable + @Unstable public abstract void setIdentifier(ByteBuffer identifier); /** @@ -65,7 +67,7 @@ public static Token newInstance(byte[] identifier, String kind, byte[] password, public abstract ByteBuffer getPassword(); @Private - @Stable + @Unstable public abstract void setPassword(ByteBuffer password); /** @@ -77,7 +79,7 @@ public static Token newInstance(byte[] identifier, String kind, byte[] password, public abstract String getKind(); @Private - @Stable + @Unstable public abstract void setKind(String kind); /** @@ -89,7 +91,7 @@ public static Token newInstance(byte[] identifier, String kind, byte[] password, public abstract String getService(); @Private - @Stable + @Unstable public abstract void setService(String service); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java index 0a6b715..769494e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/URL.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.api.records; import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.yarn.util.Records; @@ -27,7 +26,7 @@ *

        URL represents a serializable {@link java.net.URL}.

        */ @Public -@Evolving +@Stable public abstract class URL { @Public @@ -46,7 +45,7 @@ public static URL newInstance(String scheme, String host, int port, String file) * @return scheme of the URL */ @Public - @Evolving + @Stable public abstract String getScheme(); /** @@ -54,7 +53,7 @@ public static URL newInstance(String scheme, String host, int port, String file) * @param scheme scheme of the URL */ @Public - @Evolving + @Stable public abstract void setScheme(String scheme); /** @@ -62,7 +61,7 @@ public static URL newInstance(String scheme, String host, int port, String file) * @return host of the URL */ @Public - @Evolving + @Stable public abstract String getHost(); /** @@ -70,7 +69,7 @@ public static URL newInstance(String scheme, String host, int port, String file) * @param host host of the URL */ @Public - @Evolving + @Stable public abstract void setHost(String host); /** @@ -78,7 +77,7 @@ public static URL newInstance(String scheme, String host, int port, String file) * @return port of the URL */ @Public - @Evolving + @Stable public abstract int getPort(); /** @@ -86,7 +85,7 @@ public static URL newInstance(String scheme, String host, int port, String file) * @param port port of the URL */ @Public - @Evolving + @Stable public abstract void setPort(int port); /** @@ -94,7 +93,7 @@ public static URL newInstance(String scheme, String host, int port, String file) * @return file of the URL */ @Public - @Evolving + @Stable public abstract String getFile(); /** @@ -102,6 +101,6 @@ public static URL newInstance(String scheme, String host, int port, String file) * @param file file of the URL */ @Public - @Evolving + @Stable public abstract void setFile(String file); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationState.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationState.java index edbd34d..9421ce9 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationState.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationState.java @@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability.Stable; /** - * Ennumeration of various states of an ApplicationMaster. + * Enumeration of various states of an ApplicationMaster. */ @Public @Stable diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java index 5d0565c..6610fdc 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnClusterMetrics.java @@ -34,6 +34,7 @@ public abstract class YarnClusterMetrics { @Private + @Unstable public static YarnClusterMetrics newInstance(int numNodeManagers) { YarnClusterMetrics metrics = Records.newRecord(YarnClusterMetrics.class); metrics.setNumNodeManagers(numNodeManagers); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java index 6cf3d03..394d50b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; @@ -25,6 +26,7 @@ import com.google.common.base.Preconditions; +@Private public class ApplicationAttemptIdPBImpl extends ApplicationAttemptId { ApplicationAttemptIdProto proto = null; ApplicationAttemptIdProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java index c758f54..850650f 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationIdPBImpl.java @@ -19,12 +19,13 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; import com.google.common.base.Preconditions; - +@Private public class ApplicationIdPBImpl extends ApplicationId { ApplicationIdProto proto = null; ApplicationIdProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java index ad8a791..9395d97 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -35,6 +36,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto; import org.apache.hadoop.yarn.util.ProtoUtils; +@Private public class ApplicationReportPBImpl extends ApplicationReport { ApplicationReportProto proto = ApplicationReportProto.getDefaultInstance(); ApplicationReportProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationResourceUsageReportPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationResourceUsageReportPBImpl.java index 9fa8e9d..0f63a4b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationResourceUsageReportPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationResourceUsageReportPBImpl.java @@ -18,12 +18,14 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProto; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProtoOrBuilder; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; +@Private public class ApplicationResourceUsageReportPBImpl extends ApplicationResourceUsageReport { ApplicationResourceUsageReportProto proto = diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java index 0968aaf..9a8b95b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; @@ -29,7 +30,8 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto; import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; - + +@Private public class ApplicationSubmissionContextPBImpl extends ApplicationSubmissionContext { ApplicationSubmissionContextProto proto = diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java index 7049260..fa64502 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; @@ -25,7 +26,7 @@ import com.google.common.base.Preconditions; - +@Private public class ContainerIdPBImpl extends ContainerId { ContainerIdProto proto = null; ContainerIdProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java index 667e09b..c4a88b6 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.LocalResource; @@ -39,6 +40,7 @@ import com.google.protobuf.ByteString; +@Private public class ContainerLaunchContextPBImpl extends ContainerLaunchContext { ContainerLaunchContextProto proto = diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java index 22bac02..29dc1af 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -31,7 +32,8 @@ import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; - + +@Private public class ContainerPBImpl extends Container { ContainerProto proto = ContainerProto.getDefaultInstance(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java index 5c0a1c0..2ed4954 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; @@ -28,8 +29,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProtoOrBuilder; import org.apache.hadoop.yarn.util.ProtoUtils; - - +@Private public class ContainerStatusPBImpl extends ContainerStatus { ContainerStatusProto proto = ContainerStatusProto.getDefaultInstance(); ContainerStatusProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java index 4dd77bc..8313351 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/LocalResourcePBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; @@ -29,8 +30,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.URLProto; import org.apache.hadoop.yarn.util.ProtoUtils; - - +@Private public class LocalResourcePBImpl extends LocalResource { LocalResourceProto proto = LocalResourceProto.getDefaultInstance(); LocalResourceProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NMTokenPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NMTokenPBImpl.java index 28876aa..5b8fb04 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NMTokenPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NMTokenPBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.NodeId; @@ -26,7 +27,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.NMTokenProto; import org.apache.hadoop.yarn.proto.YarnServiceProtos.NMTokenProtoOrBuilder; - +@Private public class NMTokenPBImpl extends NMToken{ NMTokenProto proto = NMTokenProto.getDefaultInstance(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeIdPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeIdPBImpl.java index d406b10..85eab71 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeIdPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeIdPBImpl.java @@ -19,13 +19,13 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; import com.google.common.base.Preconditions; - - +@Private public class NodeIdPBImpl extends NodeId { NodeIdProto proto = null; NodeIdProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java index 8e8f877..2c203c4 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeReportPBImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.NodeState; @@ -28,6 +29,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; import org.apache.hadoop.yarn.util.ProtoUtils; +@Private public class NodeReportPBImpl extends NodeReport { private NodeReportProto proto = NodeReportProto.getDefaultInstance(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContainerPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContainerPBImpl.java index a67e37a..ecac147 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContainerPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContainerPBImpl.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.PreemptionContainer; import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContainerProto; import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContainerProtoOrBuilder; +@Private public class PreemptionContainerPBImpl extends PreemptionContainer { PreemptionContainerProto proto = diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java index e6b97c3..c9ff940 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionContractPBImpl.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Set; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.PreemptionContainer; import org.apache.hadoop.yarn.api.records.PreemptionContract; import org.apache.hadoop.yarn.api.records.PreemptionResourceRequest; @@ -31,6 +32,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContractProtoOrBuilder; import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionResourceRequestProto; +@Private public class PreemptionContractPBImpl extends PreemptionContract { PreemptionContractProto proto = PreemptionContractProto.getDefaultInstance(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionMessagePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionMessagePBImpl.java index 1163b00..1edc164 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionMessagePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionMessagePBImpl.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.PreemptionContract; import org.apache.hadoop.yarn.api.records.PreemptionMessage; import org.apache.hadoop.yarn.api.records.StrictPreemptionContract; @@ -25,6 +26,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProtoOrBuilder; import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto; +@Private public class PreemptionMessagePBImpl extends PreemptionMessage { PreemptionMessageProto proto = PreemptionMessageProto.getDefaultInstance(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionResourceRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionResourceRequestPBImpl.java index 3297fe5..c58585a 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionResourceRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PreemptionResourceRequestPBImpl.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.PreemptionResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionResourceRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionResourceRequestProtoOrBuilder; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto; +@Private public class PreemptionResourceRequestPBImpl extends PreemptionResourceRequest { PreemptionResourceRequestProto proto = diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java index 3c67d66..2366aa5 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/PriorityPBImpl.java @@ -18,10 +18,12 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto; import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProtoOrBuilder; - + +@Private public class PriorityPBImpl extends Priority { PriorityProto proto = PriorityProto.getDefaultInstance(); PriorityProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java index 3c24c34..3cc4ca3 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueInfoPBImpl.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.List; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.QueueInfo; import org.apache.hadoop.yarn.api.records.QueueState; @@ -31,6 +32,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto; import org.apache.hadoop.yarn.util.ProtoUtils; +@Private public class QueueInfoPBImpl extends QueueInfo { QueueInfoProto proto = QueueInfoProto.getDefaultInstance(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java index a34eaa1..69587c9 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueUserACLInfoPBImpl.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.List; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto; @@ -29,6 +30,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.QueueUserACLInfoProtoOrBuilder; import org.apache.hadoop.yarn.util.ProtoUtils; +@Private public class QueueUserACLInfoPBImpl extends QueueUserACLInfo { QueueUserACLInfoProto proto = QueueUserACLInfoProto.getDefaultInstance(); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java index 583068b..5508d2d 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceBlacklistRequestPBImpl.java @@ -21,10 +21,12 @@ import java.util.ArrayList; import java.util.List; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProtoOrBuilder; +@Private public class ResourceBlacklistRequestPBImpl extends ResourceBlacklistRequest { ResourceBlacklistRequestProto proto = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java index 276f395..83c0743 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java @@ -19,10 +19,12 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProtoOrBuilder; +@Private public class ResourcePBImpl extends Resource { ResourceProto proto = ResourceProto.getDefaultInstance(); ResourceProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java index 2b85acb..192b4b3 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourceRequestPBImpl.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; @@ -27,6 +28,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProtoOrBuilder; +@Private public class ResourceRequestPBImpl extends ResourceRequest { ResourceRequestProto proto = ResourceRequestProto.getDefaultInstance(); ResourceRequestProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java index 4e06b87..283d2b3 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/StrictPreemptionContractPBImpl.java @@ -22,12 +22,14 @@ import java.util.List; import java.util.Set; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.PreemptionContainer; import org.apache.hadoop.yarn.api.records.StrictPreemptionContract; import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContainerProto; import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto; import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProtoOrBuilder; +@Private public class StrictPreemptionContractPBImpl extends StrictPreemptionContract { StrictPreemptionContractProto proto = diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/TokenPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/TokenPBImpl.java index b58df82..1fecb69 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/TokenPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/TokenPBImpl.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.proto.SecurityProtos.TokenProtoOrBuilder; import org.apache.hadoop.yarn.api.records.Token; @@ -27,6 +28,7 @@ import com.google.protobuf.ByteString; +@Private public class TokenPBImpl extends Token { private TokenProto proto = TokenProto.getDefaultInstance(); private TokenProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/URLPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/URLPBImpl.java index 4d50776..3a68765 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/URLPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/URLPBImpl.java @@ -19,12 +19,12 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.proto.YarnProtos.URLProto; import org.apache.hadoop.yarn.proto.YarnProtos.URLProtoOrBuilder; - - +@Private public class URLPBImpl extends URL { URLProto proto = URLProto.getDefaultInstance(); URLProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/YarnClusterMetricsPBImpl.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/YarnClusterMetricsPBImpl.java index bd31db0..c1953c3 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/YarnClusterMetricsPBImpl.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/YarnClusterMetricsPBImpl.java @@ -19,12 +19,12 @@ package org.apache.hadoop.yarn.api.records.impl.pb; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.proto.YarnProtos.YarnClusterMetricsProto; import org.apache.hadoop.yarn.proto.YarnProtos.YarnClusterMetricsProtoOrBuilder; - - +@Private public class YarnClusterMetricsPBImpl extends YarnClusterMetrics { YarnClusterMetricsProto proto = YarnClusterMetricsProto.getDefaultInstance(); YarnClusterMetricsProto.Builder builder = null; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/package-info.java new file mode 100644 index 0000000..b6db540 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ +@InterfaceAudience.Public +package org.apache.hadoop.yarn.api.records; +import org.apache.hadoop.classification.InterfaceAudience; + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index d6cb329..56f3b7b 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -23,6 +23,8 @@ import java.net.UnknownHostException; import java.util.Arrays; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.StringUtils; @@ -30,6 +32,8 @@ import com.google.common.base.Joiner; +@Public +@Evolving public class YarnConfiguration extends Configuration { private static final Joiner JOINER = Joiner.on(""); diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/package-info.java new file mode 100644 index 0000000..ca0aa31 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ +@InterfaceAudience.Public +package org.apache.hadoop.yarn.conf; +import org.apache.hadoop.classification.InterfaceAudience; + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnException.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnException.java index 54b6797..c1e2b8d 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnException.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnException.java @@ -18,6 +18,15 @@ package org.apache.hadoop.yarn.exceptions; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; + +/** + * YarnException indicates exceptions from yarn servers. On the other hand, + * IOExceptions indicates exceptions from RPC layer. + */ +@Public +@Stable public class YarnException extends Exception { private static final long serialVersionUID = 1L; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnRuntimeException.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnRuntimeException.java index 968d7bf..5364a89 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnRuntimeException.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/YarnRuntimeException.java @@ -18,12 +18,17 @@ package org.apache.hadoop.yarn.exceptions; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; + /** Base Yarn Exception. * * NOTE: All derivatives of this exception, which may be thrown by a remote * service, must include a String only constructor for the exception to be * unwrapped on the client. */ +@Public +@Stable public class YarnRuntimeException extends RuntimeException { private static final long serialVersionUID = -7153142425412203936L; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factories/RecordFactory.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factories/RecordFactory.java index 219502d..d7db959 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factories/RecordFactory.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factories/RecordFactory.java @@ -18,8 +18,9 @@ package org.apache.hadoop.yarn.factories; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; - +@LimitedPrivate({ "MapReduce", "YARN" }) public interface RecordFactory { public T newRecordInstance(Class clazz); } diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factories/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factories/package-info.java new file mode 100644 index 0000000..cf639e4 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factories/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ +@InterfaceAudience.Private +package org.apache.hadoop.yarn.factories; +import org.apache.hadoop.classification.InterfaceAudience; + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java index e04104c..d90a78c 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/RecordFactoryProvider.java @@ -21,11 +21,13 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; +@LimitedPrivate({ "MapReduce", "YARN" }) public class RecordFactoryProvider { private static Configuration defaultConf; diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java new file mode 100644 index 0000000..4f397f3 --- /dev/null +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/factory/providers/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ +@InterfaceAudience.Private +package org.apache.hadoop.yarn.factory.providers; +import org.apache.hadoop.classification.InterfaceAudience; + diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java index bc8b04c..d04606e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java @@ -21,6 +21,7 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.token.Token; @@ -54,6 +55,7 @@ import com.google.protobuf.ByteString; +@Private public class ProtoUtils { diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/Records.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/Records.java index 60711fd..0d2e2df 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/Records.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/Records.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.yarn.util; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; /** * Convenient API record utils */ +@LimitedPrivate({ "MapReduce", "YARN" }) public class Records { // The default record factory private static final RecordFactory factory = diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java index 0906346..949700e 100644 --- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java +++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClient.java @@ -53,7 +53,7 @@ * AMRMClient.createAMRMClientContainerRequest(appAttemptId) * } * @param appAttemptId the appAttemptId associated with the AMRMClient - * @return the newly create AMRMClient instance. + * @return the newly created AMRMClient instance. */ @Public public static AMRMClient createAMRMClient(