YARN-6903. Yarn-native-service framework core rewrite. Contributed by Jian He

This commit is contained in:
Billie Rinaldi 2017-08-14 14:12:12 -07:00 committed by Jian He
parent 27dcc67959
commit 1888318c89
232 changed files with 6406 additions and 2074 deletions

View File

@ -49,9 +49,9 @@ function hadoop_usage
hadoop_add_subcommand "router" daemon "run the Router daemon"
hadoop_add_subcommand "schedulerconf" client "Updates scheduler configuration"
hadoop_add_subcommand "scmadmin" admin "SharedCacheManager admin tools"
hadoop_add_subcommand "servicesapi" "run slider services api"
hadoop_add_subcommand "servicesapi" "run yarn-service rest server"
hadoop_add_subcommand "sharedcachemanager" daemon "run the SharedCacheManager daemon"
hadoop_add_subcommand "slider" "run a slider app"
hadoop_add_subcommand "service" "run a service"
hadoop_add_subcommand "timelinereader" client "run the timeline reader server"
hadoop_add_subcommand "timelineserver" daemon "run the timeline server"
hadoop_add_subcommand "top" client "view cluster information"
@ -170,9 +170,9 @@ ${HADOOP_COMMON_HOME}/${HADOOP_COMMON_LIB_JARS_DIR}"
HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.sharedcachemanager.SharedCacheManager'
;;
slider)
service)
hadoop_add_classpath "${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider"'/*'
HADOOP_CLASSNAME='org.apache.slider.Slider'
HADOOP_CLASSNAME='org.apache.hadoop.yarn.service.client.ServiceCLI'
local sld="${HADOOP_YARN_HOME}/${YARN_DIR},\
${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR},\
${HADOOP_YARN_HOME}/${YARN_LIB_JARS_DIR}/slider,\

View File

@ -18,23 +18,21 @@
package org.apache.hadoop.yarn.services.api.impl;
import com.google.inject.Singleton;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.service.client.ServiceClient;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.ApplicationState;
import org.apache.slider.api.resource.ApplicationStatus;
import org.apache.slider.api.resource.Component;
import org.apache.slider.util.ServiceApiUtil;
import org.apache.slider.client.SliderClient;
import org.apache.slider.common.params.ActionFreezeArgs;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.common.tools.SliderVersionInfo;
import org.apache.slider.core.buildutils.BuildHelper;
import org.apache.slider.core.exceptions.SliderException;
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -51,7 +49,6 @@
import javax.ws.rs.core.Response.Status;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static org.apache.slider.util.RestApiConstants.*;
@ -61,51 +58,39 @@
@Consumes({ MediaType.APPLICATION_JSON })
@Produces({ MediaType.APPLICATION_JSON })
public class ApplicationApiService {
private static final Logger logger =
private static final Logger LOG =
LoggerFactory.getLogger(ApplicationApiService.class);
private static org.apache.hadoop.conf.Configuration SLIDER_CONFIG =
new YarnConfiguration();
private static SliderClient SLIDER_CLIENT;
private static Response SLIDER_VERSION;
private static final ActionFreezeArgs ACTION_FREEZE_ARGS = new ActionFreezeArgs();
private static Configuration YARN_CONFIG = new YarnConfiguration();
private static ServiceClient SERVICE_CLIENT;
static {
init();
}
// initialize all the common resources - order is important
protected static void init() {
SLIDER_CLIENT = createSliderClient();
SLIDER_VERSION = initSliderVersion();
private static void init() {
SERVICE_CLIENT = new ServiceClient();
SERVICE_CLIENT.init(YARN_CONFIG);
SERVICE_CLIENT.start();
}
@GET
@Path("/versions/slider-version")
@Path("/versions/yarn-service-version")
@Consumes({ MediaType.APPLICATION_JSON })
@Produces({ MediaType.APPLICATION_JSON }) public Response getSliderVersion() {
logger.info("GET: getSliderVersion");
return SLIDER_VERSION;
}
private static Response initSliderVersion() {
Map<String, Object> metadata = new HashMap<>();
BuildHelper.addBuildMetadata(metadata, "org.apache.hadoop.yarn.services");
String sliderVersion = metadata.toString();
logger.info("Slider version = {}", sliderVersion);
String hadoopVersion = SliderVersionInfo.getHadoopVersionString();
logger.info("Hadoop version = {}", hadoopVersion);
return Response.ok("{ \"slider_version\": \"" + sliderVersion
+ "\", \"hadoop_version\": \"" + hadoopVersion + "\"}").build();
String version = VersionInfo.getBuildVersion();
LOG.info(version);
return Response.ok(version).build();
}
@POST @Consumes({ MediaType.APPLICATION_JSON })
@Produces({ MediaType.APPLICATION_JSON })
public Response createApplication(Application application) {
logger.info("POST: createApplication = {}", application);
LOG.info("POST: createApplication = {}", application);
ApplicationStatus applicationStatus = new ApplicationStatus();
try {
ApplicationId applicationId = SLIDER_CLIENT.actionCreate(application);
logger.info("Successfully created application " + application.getName()
ApplicationId applicationId = SERVICE_CLIENT.actionCreate(application);
LOG.info("Successfully created application " + application.getName()
+ " applicationId = " + applicationId);
applicationStatus.setState(ApplicationState.ACCEPTED);
applicationStatus.setUri(
@ -118,58 +103,18 @@ public Response createApplication(Application application) {
.build();
} catch (Exception e) {
String message = "Failed to create application " + application.getName();
logger.error(message, e);
LOG.error(message, e);
applicationStatus.setDiagnostics(message + ": " + e.getMessage());
return Response.status(Status.INTERNAL_SERVER_ERROR)
.entity(applicationStatus).build();
}
}
protected static SliderClient createSliderClient() {
if (SLIDER_CLIENT != null) {
return SLIDER_CLIENT;
}
org.apache.hadoop.conf.Configuration sliderClientConfiguration =
SLIDER_CONFIG;
SliderClient client = new SliderClient() {
@Override public void init(org.apache.hadoop.conf.Configuration conf) {
super.init(conf);
try {
initHadoopBinding();
} catch (SliderException | IOException e) {
throw new RuntimeException(
"Unable to automatically init Hadoop binding", e);
}
}
};
try {
logger
.debug("Slider Client configuration: {}", sliderClientConfiguration);
sliderClientConfiguration = client.bindArgs(sliderClientConfiguration, new String[] { "help" });
client.init(sliderClientConfiguration);
client.start();
} catch (Exception e) {
logger.error("Unable to create SliderClient", e);
throw new RuntimeException(e.getMessage(), e);
}
return client;
}
// The information this REST endpoint currently returned can be retrieved from
// RM web services
// Probably the data from AM is more important. Do that later.
// @GET @Consumes({ MediaType.APPLICATION_JSON })
// @Produces({ MediaType.APPLICATION_JSON })
// public Response getApplications(@QueryParam("state") String state) {
// logger.info("GET: getApplications with param state = {}", state);
// return null;
// }
@GET @Path("/{app_name}")
@Consumes({ MediaType.APPLICATION_JSON })
@Produces({ MediaType.APPLICATION_JSON })
public Response getApplication(@PathParam("app_name") String appName) {
logger.info("GET: getApplication for appName = {}", appName);
LOG.info("GET: getApplication for appName = {}", appName);
ApplicationStatus applicationStatus = new ApplicationStatus();
// app name validation
@ -181,24 +126,25 @@ public Response getApplication(@PathParam("app_name") String appName) {
}
try {
Application app = SLIDER_CLIENT.actionStatus(appName);
ApplicationReport report = SLIDER_CLIENT.findInstance(appName);
if (app != null && report != null) {
Application app = SERVICE_CLIENT.getStatus(appName);
ApplicationReport report = SERVICE_CLIENT.getYarnClient()
.getApplicationReport(ApplicationId.fromString(app.getId()));
if (report != null) {
app.setLifetime(
report.getApplicationTimeouts().get(ApplicationTimeoutType.LIFETIME)
.getRemainingTime());
logger.info("Application = {}", app);
LOG.info("Application = {}", app);
return Response.ok(app).build();
} else {
String message = "Application " + appName + " does not exist.";
logger.info(message);
LOG.info(message);
applicationStatus.setCode(ERROR_CODE_APP_DOES_NOT_EXIST);
applicationStatus.setDiagnostics(message);
return Response.status(Status.NOT_FOUND).entity(applicationStatus)
.build();
}
} catch (Exception e) {
logger.error("Get application failed", e);
LOG.error("Get application failed", e);
applicationStatus
.setDiagnostics("Failed to retrieve application: " + e.getMessage());
return Response.status(Status.INTERNAL_SERVER_ERROR)
@ -211,18 +157,18 @@ public Response getApplication(@PathParam("app_name") String appName) {
@Consumes({ MediaType.APPLICATION_JSON })
@Produces({ MediaType.APPLICATION_JSON })
public Response deleteApplication(@PathParam("app_name") String appName) {
logger.info("DELETE: deleteApplication for appName = {}", appName);
LOG.info("DELETE: deleteApplication for appName = {}", appName);
return stopApplication(appName, true);
}
private Response stopApplication(String appName, boolean destroy) {
try {
SLIDER_CLIENT.actionStop(appName, ACTION_FREEZE_ARGS);
SERVICE_CLIENT.actionStop(appName);
if (destroy) {
SLIDER_CLIENT.actionDestroy(appName);
logger.info("Successfully deleted application {}", appName);
SERVICE_CLIENT.actionDestroy(appName);
LOG.info("Successfully deleted application {}", appName);
} else {
logger.info("Successfully stopped application {}", appName);
LOG.info("Successfully stopped application {}", appName);
}
return Response.status(Status.NO_CONTENT).build();
} catch (ApplicationNotFoundException e) {
@ -252,8 +198,8 @@ public Response updateComponent(@PathParam("app_name") String appName,
.getNumberOfContainers()).build();
}
try {
Map<String, Long> original = SLIDER_CLIENT.flex(appName, Collections
.singletonMap(component.getName(),
Map<String, Long> original = SERVICE_CLIENT.flexByRestService(appName,
Collections.singletonMap(component.getName(),
component.getNumberOfContainers()));
return Response.ok().entity("Updating " + componentName + " size from "
+ original.get(componentName) + " to "
@ -271,7 +217,7 @@ public Response updateComponent(@PathParam("app_name") String appName,
@Produces({ MediaType.APPLICATION_JSON })
public Response updateApplication(@PathParam("app_name") String appName,
Application updateAppData) {
logger.info("PUT: updateApplication for app = {} with data = {}", appName,
LOG.info("PUT: updateApplication for app = {} with data = {}", appName,
updateAppData);
// Ignore the app name provided in updateAppData and always use appName
@ -314,14 +260,14 @@ public Response updateApplication(@PathParam("app_name") String appName,
private Response updateLifetime(String appName, Application updateAppData) {
try {
String newLifeTime =
SLIDER_CLIENT.updateLifetime(appName, updateAppData.getLifetime());
SERVICE_CLIENT.updateLifetime(appName, updateAppData.getLifetime());
return Response.ok("Application " + appName + " lifeTime is successfully updated to "
+ updateAppData.getLifetime() + " seconds from now: " + newLifeTime).build();
} catch (Exception e) {
String message =
"Failed to update application (" + appName + ") lifetime ("
+ updateAppData.getLifetime() + ")";
logger.error(message, e);
LOG.error(message, e);
return Response.status(Status.INTERNAL_SERVER_ERROR)
.entity(message + " : " + e.getMessage()).build();
}
@ -329,17 +275,12 @@ private Response updateLifetime(String appName, Application updateAppData) {
private Response startApplication(String appName) {
try {
int ret = SLIDER_CLIENT.actionList(appName);
if (ret == 0) {
return Response.ok()
.entity("Application " + appName + " is already alive.").build();
}
SLIDER_CLIENT.actionStart(appName, null);
logger.info("Successfully started application " + appName);
SERVICE_CLIENT.actionStart(appName);
LOG.info("Successfully started application " + appName);
return Response.ok("Application " + appName + " is successfully started").build();
} catch (Exception e) {
String message = "Failed to start application " + appName;
logger.info(message, e);
LOG.info(message, e);
return Response.status(Status.INTERNAL_SERVER_ERROR)
.entity(message + ": " + e.getMessage()).build();
}

View File

@ -16,6 +16,14 @@
limitations under the License.
-->
<FindBugsFilter>
<Match>
<Package name="org.apache.hadoop.yarn.proto" />
</Match>
<Match>
<class name="org.apache.hadoop.yarn.service.utils.ServiceApiUtil" />
<Bug pattern="MS_CANNOT_BE_FINAL" />
</Match>
<Match>
<Package name="org.apache.slider.api.proto" />
</Match>

View File

@ -61,6 +61,7 @@
<includes>
<include>SliderClusterMessages.proto</include>
<include>SliderClusterProtocol.proto</include>
<include>ClientAMProtocol.proto</include>
</includes>
</source>
</configuration>

View File

@ -0,0 +1,40 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto;
import java.io.IOException;
public interface ClientAMProtocol {
FlexComponentsResponseProto flexComponents(FlexComponentsRequestProto request)
throws IOException, YarnException;
GetStatusResponseProto getStatus(GetStatusRequestProto requestProto)
throws IOException, YarnException;
StopResponseProto stop(StopRequestProto requestProto)
throws IOException, YarnException;
}

View File

@ -0,0 +1,132 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.ComponentCountProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto;
import org.apache.hadoop.yarn.service.component.ComponentEvent;
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.InetSocketAddress;
import static org.apache.hadoop.yarn.service.component.ComponentEventType.FLEX;
public class ClientAMService extends AbstractService
implements ClientAMProtocol {
private static final Logger LOG =
LoggerFactory.getLogger(ClientAMService.class);
private ServiceContext context;
private Server server;
private InetSocketAddress bindAddress;
public ClientAMService(ServiceContext context) {
super("Client AM Service");
this.context = context;
}
@Override protected void serviceStart() throws Exception {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
InetSocketAddress address = new InetSocketAddress(0);
server = rpc.getServer(ClientAMProtocol.class, this, address, conf,
context.secretManager, 1);
server.start();
String nodeHostString =
System.getenv(ApplicationConstants.Environment.NM_HOST.name());
bindAddress = NetUtils.createSocketAddrForHost(nodeHostString,
server.getListenerAddress().getPort());
LOG.info("Instantiated ClientAMService at " + bindAddress);
super.serviceStart();
}
@Override protected void serviceStop() throws Exception {
if (server != null) {
server.stop();
}
super.serviceStop();
}
@Override public FlexComponentsResponseProto flexComponents(
FlexComponentsRequestProto request) throws IOException {
if (!request.getComponentsList().isEmpty()) {
for (ComponentCountProto component : request.getComponentsList()) {
ComponentEvent event = new ComponentEvent(component.getName(), FLEX)
.setDesired(component.getNumberOfContainers());
context.scheduler.getDispatcher().getEventHandler().handle(event);
LOG.info("Flexing component {} to {}", component.getName(),
component.getNumberOfContainers());
}
}
return FlexComponentsResponseProto.newBuilder().build();
}
@Override
public GetStatusResponseProto getStatus(GetStatusRequestProto request)
throws IOException, YarnException {
String stat = ServiceApiUtil.jsonSerDeser.toJson(context.application);
return GetStatusResponseProto.newBuilder().setStatus(stat).build();
}
@Override
public StopResponseProto stop(StopRequestProto requestProto)
throws IOException, YarnException {
LOG.info("Stop the service.");
// Stop the service in 2 seconds delay to make sure this rpc call is completed.
// shutdown hook will be executed which will stop AM gracefully.
Thread thread = new Thread() {
@Override
public void run() {
try {
Thread.sleep(2000);
ExitUtil.terminate(0);
} catch (InterruptedException e) {
LOG.error("Interrupted while stopping", e);
}
}
};
thread.start();
return StopResponseProto.newBuilder().build();
}
public InetSocketAddress getBindAddress() {
return bindAddress;
}
}

View File

@ -0,0 +1,89 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import org.apache.hadoop.yarn.service.component.Component;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.apache.slider.api.ResourceKeys.NODE_FAILURE_THRESHOLD;
/**
* This tracks the container failures per node. If the failure counter exceeds
* the maxFailurePerNode limit, it'll blacklist that node.
*
*/
public class ContainerFailureTracker {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerFailureTracker.class);
// Host -> num container failures
private Map<String, Integer> failureCountPerNode = new HashMap<>();
private Set<String> blackListedNodes = new HashSet<>();
private ServiceContext context;
private int maxFailurePerNode;
private Component component;
public ContainerFailureTracker(ServiceContext context, Component component) {
this.context = context;
this.component = component;
maxFailurePerNode = component.getComponentSpec().getConfiguration()
.getPropertyInt(NODE_FAILURE_THRESHOLD, 3);
}
public synchronized void incNodeFailure(String host) {
int num = 0;
if (failureCountPerNode.containsKey(host)) {
num = failureCountPerNode.get(host);
}
num++;
failureCountPerNode.put(host, num);
// black list the node if exceed max failure
if (num > maxFailurePerNode && !blackListedNodes.contains(host)) {
List<String> blacklists = new ArrayList<>();
blacklists.add(host);
blackListedNodes.add(host);
context.scheduler.getAmRMClient().updateBlacklist(blacklists, null);
LOG.info("[COMPONENT {}]: Failed {} times on this host, blacklisted {}."
+ " Current list of blacklisted nodes: {}",
component.getName(), num, host, blackListedNodes);
}
}
public synchronized void resetContainerFailures() {
// reset container failure counter per node
failureCountPerNode.clear();
context.scheduler.getAmRMClient()
.updateBlacklist(null, new ArrayList<>(blackListedNodes));
LOG.info("[COMPONENT {}]: Clearing blacklisted nodes {} ",
component.getName(), blackListedNodes);
blackListedNodes.clear();
}
}

View File

@ -0,0 +1,101 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.provider.ProviderService;
import org.apache.hadoop.yarn.service.provider.ProviderFactory;
import org.apache.slider.api.resource.Application;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.core.launch.AbstractLauncher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
public class ContainerLaunchService extends AbstractService{
protected static final Logger LOG =
LoggerFactory.getLogger(ContainerLaunchService.class);
private ExecutorService executorService;
private SliderFileSystem fs;
public ContainerLaunchService(SliderFileSystem fs) {
super(ContainerLaunchService.class.getName());
this.fs = fs;
}
@Override
public void serviceInit(Configuration conf) throws Exception {
executorService = Executors.newCachedThreadPool();
super.serviceInit(conf);
}
@Override
protected void serviceStop() throws Exception {
if (executorService != null) {
executorService.shutdownNow();
}
super.serviceStop();
}
public void launchCompInstance(Application application,
ComponentInstance instance, Container container) {
ContainerLauncher launcher =
new ContainerLauncher(application, instance, container);
executorService.execute(launcher);
}
private class ContainerLauncher implements Runnable {
public final Container container;
public final Application application;
public ComponentInstance instance;
public ContainerLauncher(
Application application,
ComponentInstance instance, Container container) {
this.container = container;
this.application = application;
this.instance = instance;
}
@Override public void run() {
org.apache.slider.api.resource.Component compSpec = instance.getCompSpec();
ProviderService provider = ProviderFactory.getProviderService(
compSpec.getArtifact());
AbstractLauncher launcher = new AbstractLauncher(fs, null);
try {
provider.buildContainerLaunchContext(launcher, application,
instance, fs);
instance.getComponent().getScheduler().getNmClient()
.startContainerAsync(container,
launcher.completeContainerLaunch());
} catch (Exception e) {
LOG.error(instance.getCompInstanceId()
+ ": Failed to launch container. ", e);
}
}
}
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import com.google.common.cache.LoadingCache;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.ConfigFile;
import org.apache.slider.common.tools.SliderFileSystem;
public class ServiceContext {
public Application application = null;
public SliderFileSystem fs;
public String serviceHdfsDir = "";
public ApplicationAttemptId attemptId;
public LoadingCache<ConfigFile, Object> configCache;
public ServiceScheduler scheduler;
public ClientToAMTokenSecretManager secretManager;
public ClientAMService clientAMService;
public ServiceContext() {
}
}

View File

@ -0,0 +1,132 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
import org.apache.hadoop.yarn.service.client.params.SliderAMArgs;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
public class ServiceMaster extends CompositeService {
private static final Logger LOG =
LoggerFactory.getLogger(ServiceMaster.class);
private static SliderAMArgs amArgs;
public ServiceMaster(String name) {
super(name);
}
@Override
protected void serviceInit(Configuration conf) throws Exception {
//TODO Deprecate slider conf, make sure works with yarn conf
printSystemEnv();
if (UserGroupInformation.isSecurityEnabled()) {
UserGroupInformation.setConfiguration(conf);
}
LOG.info("Login user is {}", UserGroupInformation.getLoginUser());
ServiceContext context = new ServiceContext();
Path appDir = new Path(amArgs.getAppDefPath()).getParent();
SliderFileSystem fs = new SliderFileSystem(conf);
context.fs = fs;
fs.setAppDir(appDir);
context.application = ServiceApiUtil
.loadApplicationFrom(fs, new Path(amArgs.getAppDefPath()));
LOG.info(context.application.toString());
ContainerId amContainerId = ContainerId.fromString(SliderUtils
.mandatoryEnvVariable(
ApplicationConstants.Environment.CONTAINER_ID.name()));
ApplicationAttemptId attemptId = amContainerId.getApplicationAttemptId();
LOG.info("Application attemptId: " + attemptId);
context.attemptId = attemptId;
// configure AM to wait forever for RM
conf.setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, -1);
conf.unset(YarnConfiguration.CLIENT_FAILOVER_MAX_ATTEMPTS);
DefaultMetricsSystem.initialize("ServiceAppMaster");
context.secretManager = new ClientToAMTokenSecretManager(attemptId, null);
ClientAMService clientAMService = new ClientAMService(context);
context.clientAMService = clientAMService;
addService(clientAMService);
ServiceScheduler scheduler = new ServiceScheduler(context);
addService(scheduler);
context.scheduler = scheduler;
ServiceMonitor monitor = new ServiceMonitor("Service Monitor", context);
addService(monitor);
super.serviceInit(conf);
}
@Override
protected void serviceStop() throws Exception {
LOG.info("Stopping app master");
super.serviceStop();
}
private void printSystemEnv() {
for (Map.Entry<String, String> envs : System.getenv().entrySet()) {
LOG.info("{} = {}", envs.getKey(), envs.getValue());
}
}
public static void main(String[] args) throws Exception {
Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
StringUtils.startupShutdownMessage(ServiceMaster.class, args, LOG);
amArgs = new SliderAMArgs(args);
amArgs.parse();
try {
ServiceMaster serviceMaster = new ServiceMaster("Service Master");
ShutdownHookManager.get()
.addShutdownHook(new CompositeServiceShutdownHook(serviceMaster), 30);
YarnConfiguration conf = new YarnConfiguration();
new GenericOptionsParser(conf, args);
serviceMaster.init(conf);
serviceMaster.start();
} catch (Throwable t) {
LOG.error("Error starting service master", t);
ExitUtil.terminate(1, "Error starting service master");
}
}
}

View File

@ -0,0 +1,143 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.service.component.Component;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.slider.api.InternalKeys;
import org.apache.hadoop.yarn.service.component.ComponentEvent;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEvent;
import org.apache.hadoop.yarn.service.component.ComponentState;
import org.apache.slider.api.ResourceKeys;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.server.servicemonitor.ProbeStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.yarn.service.component.ComponentEventType.FLEX;
import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.BECOME_NOT_READY;
import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.BECOME_READY;
import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceState.READY;
public class ServiceMonitor extends AbstractService {
private static final Logger LOG =
LoggerFactory.getLogger(ServiceMonitor.class);
public ScheduledExecutorService executorService;
private Map<ContainerId, ComponentInstance> liveInstances = null;
private ServiceContext context;
public ServiceMonitor(String name, ServiceContext context) {
super(name);
liveInstances = context.scheduler.getLiveInstances();
this.context = context;
}
@Override
public void serviceInit(Configuration conf) throws Exception {
executorService = Executors.newScheduledThreadPool(1);
super.serviceInit(conf);
}
@Override
public void serviceStart() throws Exception {
long readinessCheckInterval = context.application.getConfiguration()
.getPropertyLong(InternalKeys.MONITOR_INTERVAL,
InternalKeys.DEFAULT_MONITOR_INTERVAL);
executorService
.scheduleAtFixedRate(new ReadinessChecker(), readinessCheckInterval,
readinessCheckInterval, TimeUnit.SECONDS);
long failureResetInterval = SliderUtils
.getTimeRange(context.application.getConfiguration(),
ResourceKeys.CONTAINER_FAILURE_WINDOW,
ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_DAYS,
ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_HOURS,
ResourceKeys.DEFAULT_CONTAINER_FAILURE_WINDOW_MINUTES, 0);
executorService
.scheduleAtFixedRate(new ContainerFailureReset(), failureResetInterval,
failureResetInterval, TimeUnit.SECONDS);
}
@Override
public void serviceStop() throws Exception {
if (executorService != null) {
executorService.shutdownNow();
}
}
private class ReadinessChecker implements Runnable {
@Override
public void run() {
// check if the comp instance are ready
for (Map.Entry<ContainerId, ComponentInstance> entry : liveInstances
.entrySet()) {
ComponentInstance instance = entry.getValue();
ProbeStatus status = instance.ping();
if (status.isSuccess()) {
if (instance.getState() != READY) {
// synchronously update the state.
instance.handle(
new ComponentInstanceEvent(entry.getKey(), BECOME_READY));
}
} else {
if (instance.getState() == READY) {
instance.handle(
new ComponentInstanceEvent(entry.getKey(), BECOME_NOT_READY));
}
}
}
for (Component component : context.scheduler.getAllComponents()
.values()) {
// If comp hasn't started yet and its dependencies are satisfied
if (component.getState() == ComponentState.INIT && component
.areDependenciesReady()) {
LOG.info("[COMPONENT {}]: Dependencies satisfied, ramping up.",
component.getName());
ComponentEvent event = new ComponentEvent(component.getName(), FLEX)
.setDesired(component.getComponentSpec().getNumberOfContainers());
component.handle(event);
}
}
}
}
private class ContainerFailureReset implements Runnable {
@Override
public void run() {
for (Component component : context.scheduler.getAllComponents().values()) {
component.resetCompFailureCount();
}
}
}
}

View File

@ -0,0 +1,641 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.registry.client.api.RegistryOperations;
import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
import org.apache.hadoop.registry.client.binding.RegistryUtils;
import org.apache.hadoop.registry.client.types.ServiceRecord;
import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.service.component.Component;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.util.BoundedAppender;
import org.apache.slider.api.RoleKeys;
import org.apache.slider.api.ServiceApiConstants;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.ConfigFile;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.registry.info.CustomRegistryConstants;
import org.apache.slider.core.zk.ZKIntegration;
import org.apache.hadoop.yarn.service.provider.ProviderUtils;
import org.apache.hadoop.yarn.service.metrics.ServiceMetrics;
import org.apache.hadoop.yarn.service.component.ComponentEvent;
import org.apache.hadoop.yarn.service.component.ComponentEventType;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEvent;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType;
import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher;
import org.apache.hadoop.yarn.service.timelineservice.ServiceMetricsSink;
import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
import static org.apache.slider.api.ServiceApiConstants.*;
import static org.apache.hadoop.yarn.service.component.ComponentEventType.*;
/**
*
*/
public class ServiceScheduler extends CompositeService {
private static final Logger LOG =
LoggerFactory.getLogger(ServiceScheduler.class);
private Application app;
// component_name -> component
private final Map<String, Component> componentsByName =
new ConcurrentHashMap<>();
// id - > component
private final Map<Long, Component> componentsById =
new ConcurrentHashMap<>();
private final Map<ContainerId, ComponentInstance> liveInstances =
new ConcurrentHashMap<>();
private ServiceMetrics serviceMetrics;
private ServiceTimelinePublisher serviceTimelinePublisher;
// Global diagnostics that will be reported to RM on eRxit.
// The unit the number of characters. This will be limited to 64 * 1024
// characters.
private BoundedAppender diagnostics = new BoundedAppender(64 * 1024);
// A cache for loading config files from remote such as hdfs
public LoadingCache<ConfigFile, Object> configFileCache = null;
public ScheduledExecutorService executorService;
public Map<String, String> globalTokens = new HashMap<>();
private AMRMClientAsync<AMRMClient.ContainerRequest> amRMClient;
private NMClientAsync nmClient;
private AsyncDispatcher dispatcher;
AsyncDispatcher compInstanceDispatcher;
private YarnRegistryViewForProviders yarnRegistryOperations;
private ServiceContext context;
private ContainerLaunchService containerLaunchService;
public ServiceScheduler(ServiceContext context) {
super(context.application.getName());
this.context = context;
}
public void buildInstance(ServiceContext context, Configuration configuration)
throws YarnException {
app = context.application;
executorService = Executors.newScheduledThreadPool(10);
RegistryOperations registryClient = RegistryOperationsFactory
.createInstance("ServiceScheduler", configuration);
addIfService(registryClient);
// register metrics
serviceMetrics = ServiceMetrics
.register(app.getName(), "Metrics for service");
serviceMetrics.tag("type", "Metrics type [component or service]", "service");
serviceMetrics.tag("appId", "Application id for service", app.getId());
amRMClient =
AMRMClientAsync.createAMRMClientAsync(1000, new AMRMClientCallback());
addIfService(amRMClient);
nmClient = NMClientAsync.createNMClientAsync(new NMClientCallback());
addIfService(nmClient);
dispatcher = new AsyncDispatcher("Component dispatcher");
dispatcher.register(ComponentEventType.class,
new ComponentEventHandler());
dispatcher.setDrainEventsOnStop();
addIfService(dispatcher);
compInstanceDispatcher =
new AsyncDispatcher("CompInstance dispatcher");
compInstanceDispatcher.register(ComponentInstanceEventType.class,
new ComponentInstanceEventHandler());
addIfService(compInstanceDispatcher);
containerLaunchService = new ContainerLaunchService(context.fs);
addService(containerLaunchService);
if (YarnConfiguration.timelineServiceV2Enabled(configuration)) {
TimelineV2Client timelineClient = TimelineV2Client
.createTimelineClient(context.attemptId.getApplicationId());
amRMClient.registerTimelineV2Client(timelineClient);
serviceTimelinePublisher = new ServiceTimelinePublisher(timelineClient);
addService(serviceTimelinePublisher);
DefaultMetricsSystem.instance().register("ServiceMetricsSink",
"For processing metrics to ATS",
new ServiceMetricsSink(serviceTimelinePublisher));
LOG.info("Timeline v2 is enabled.");
}
yarnRegistryOperations =
new YarnRegistryViewForProviders(registryClient,
RegistryUtils.currentUser(), SliderKeys.APP_TYPE, app.getName(),
context.attemptId);
initGlobalTokensForSubstitute(context);
//substitute quicklinks
ProviderUtils.substituteMapWithTokens(app.getQuicklinks(), globalTokens);
createConfigFileCache(context.fs.getFileSystem());
createAllComponents();
}
@Override
public void serviceInit(Configuration conf) throws Exception {
try {
buildInstance(context, conf);
} catch (YarnException e) {
throw new YarnRuntimeException(e);
}
super.serviceInit(conf);
}
@Override
public void serviceStop() throws Exception {
LOG.info("Stopping service scheduler");
if (executorService != null) {
executorService.shutdownNow();
}
DefaultMetricsSystem.shutdown();
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
serviceTimelinePublisher.serviceAttemptUnregistered(context);
}
// Cleanup each component instance. no need to release containers as
// they will be automatically released by RM
for (ComponentInstance instance : liveInstances.values()) {
instance.cleanupRegistryAndCompHdfsDir();
}
String msg = diagnostics.toString()
+ "Navigate to the failed component for more details.";
amRMClient
.unregisterApplicationMaster(FinalApplicationStatus.ENDED, msg, "");
LOG.info("Application " + app.getName()
+ " unregistered with RM, with attemptId = " + context.attemptId
+ ", diagnostics = " + diagnostics);
super.serviceStop();
}
@Override
public void serviceStart() throws Exception {
super.serviceStart();
InetSocketAddress bindAddress = context.clientAMService.getBindAddress();
RegisterApplicationMasterResponse response = amRMClient
.registerApplicationMaster(bindAddress.getHostName(),
bindAddress.getPort(), "N/A");
if (response.getClientToAMTokenMasterKey() != null
&& response.getClientToAMTokenMasterKey().remaining() != 0) {
context.secretManager
.setMasterKey(response.getClientToAMTokenMasterKey().array());
}
registerServiceInstance(context.attemptId, app);
//TODO handle containers recover
}
private void recover() {
}
private void initGlobalTokensForSubstitute(ServiceContext context) {
// ZK
globalTokens.put(ServiceApiConstants.CLUSTER_ZK_QUORUM, getConfig()
.getTrimmed(KEY_REGISTRY_ZK_QUORUM, DEFAULT_REGISTRY_ZK_QUORUM));
String user = null;
try {
user = UserGroupInformation.getCurrentUser().getShortUserName();
} catch (IOException e) {
LOG.error("Failed to get user.", e);
}
globalTokens
.put(SERVICE_ZK_PATH, ZKIntegration.mkClusterPath(user, app.getName()));
globalTokens.put(ServiceApiConstants.USER, user);
String dnsDomain = getConfig().getTrimmed(KEY_DNS_DOMAIN);
if (dnsDomain != null && !dnsDomain.isEmpty()) {
globalTokens.put(ServiceApiConstants.DOMAIN, dnsDomain);
}
// HDFS
String clusterFs = getConfig().getTrimmed(FS_DEFAULT_NAME_KEY);
if (clusterFs != null && !clusterFs.isEmpty()) {
globalTokens.put(ServiceApiConstants.CLUSTER_FS_URI, clusterFs);
globalTokens.put(ServiceApiConstants.CLUSTER_FS_HOST,
URI.create(clusterFs).getHost());
}
globalTokens.put(SERVICE_HDFS_DIR, context.serviceHdfsDir);
// service name
globalTokens.put(SERVICE_NAME_LC, app.getName().toLowerCase());
globalTokens.put(SERVICE_NAME, app.getName());
}
private void createConfigFileCache(final FileSystem fileSystem) {
this.configFileCache =
CacheBuilder.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES)
.build(new CacheLoader<ConfigFile, Object>() {
@Override public Object load(ConfigFile key) throws Exception {
switch (key.getType()) {
case HADOOP_XML:
try (FSDataInputStream input = fileSystem
.open(new Path(key.getSrcFile()))) {
org.apache.hadoop.conf.Configuration confRead =
new org.apache.hadoop.conf.Configuration(false);
confRead.addResource(input);
Map<String, String> map = new HashMap<>(confRead.size());
for (Map.Entry<String, String> entry : confRead) {
map.put(entry.getKey(), entry.getValue());
}
return map;
}
case TEMPLATE:
try (FSDataInputStream fileInput = fileSystem
.open(new Path(key.getSrcFile()))) {
return IOUtils.toString(fileInput);
}
default:
return null;
}
}
});
context.configCache = configFileCache;
}
private void registerServiceInstance(ApplicationAttemptId attemptId,
Application application) throws IOException {
LOG.info("Registering " + attemptId + ", " + application.getName()
+ " into registry");
ServiceRecord serviceRecord = new ServiceRecord();
serviceRecord.set(YarnRegistryAttributes.YARN_ID,
attemptId.getApplicationId().toString());
serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE,
PersistencePolicies.APPLICATION);
serviceRecord.description = "Slider Application Master";
serviceRecord.addExternalEndpoint(RegistryTypeUtils
.ipcEndpoint(CustomRegistryConstants.AM_IPC_PROTOCOL,
new InetSocketAddress(5000))); // FIXME
// set any provided attributes
setUserProvidedServiceRecordAttributes(application.getConfiguration(),
serviceRecord);
executorService.submit(new Runnable() {
@Override public void run() {
try {
yarnRegistryOperations.registerSelf(serviceRecord, true);
LOG.info("Registered service under {}; absolute path {}",
yarnRegistryOperations.getSelfRegistrationPath(),
yarnRegistryOperations.getAbsoluteSelfRegistrationPath());
boolean isFirstAttempt = 1 == attemptId.getAttemptId();
// delete the children in case there are any and this is an AM startup.
// just to make sure everything underneath is purged
if (isFirstAttempt) {
yarnRegistryOperations.deleteChildren(
yarnRegistryOperations.getSelfRegistrationPath(), true);
}
} catch (IOException e) {
LOG.error(
"Failed to register app " + app.getName() + " in registry");
}
}
});
if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
serviceTimelinePublisher.serviceAttemptRegistered(app);
}
}
private void setUserProvidedServiceRecordAttributes(
org.apache.slider.api.resource.Configuration conf, ServiceRecord record) {
String prefix = RoleKeys.SERVICE_RECORD_ATTRIBUTE_PREFIX;
for (Map.Entry<String, String> entry : conf.getProperties().entrySet()) {
if (entry.getKey().startsWith(prefix)) {
String key = entry.getKey().substring(prefix.length() + 1);
record.set(key, entry.getValue().trim());
}
}
}
private void createAllComponents() {
long allocateId = 0;
// sort components by dependencies
Collection<org.apache.slider.api.resource.Component> sortedComponents =
ServiceApiUtil.sortByDependencies(app.getComponents());
for (org.apache.slider.api.resource.Component compSpec : sortedComponents) {
Component component = new Component(compSpec, allocateId, context);
componentsById.put(allocateId, component);
componentsByName.put(component.getName(), component);
allocateId++;
// Trigger the component without dependencies
if (component.areDependenciesReady()) {
ComponentEvent event = new ComponentEvent(compSpec.getName(), FLEX)
.setDesired(compSpec.getNumberOfContainers());
component.handle(event);
}
}
}
private final class ComponentEventHandler
implements EventHandler<ComponentEvent> {
@Override
public void handle(ComponentEvent event) {
Component component = componentsByName.get(event.getName());
if (component == null) {
LOG.error("No component exists for " + event.getName());
return;
}
try {
component.handle(event);
} catch (Throwable t) {
LOG.error("Error in handling event type " + event.getType()
+ " for component " + event.getName(), t);
}
}
}
private final class ComponentInstanceEventHandler
implements EventHandler<ComponentInstanceEvent> {
@Override
public void handle(ComponentInstanceEvent event) {
ComponentInstance instance =
liveInstances.get(event.getContainerId());
if (instance == null) {
LOG.error("No component instance exists for " + event.getContainerId());
return;
}
try {
instance.handle(event);
} catch (Throwable t) {
LOG.error("Error in handling event type " + event.getType()
+ " for component instance " + instance.getCompInstanceId(), t);
}
}
}
private class AMRMClientCallback extends AMRMClientAsync.AbstractCallbackHandler {
@Override
public void onContainersAllocated(List<Container> containers) {
LOG.info(containers.size() + " containers allocated. ");
for (Container container : containers) {
Component comp = componentsById.get(container.getAllocationRequestId());
ComponentEvent event =
new ComponentEvent(comp.getName(), CONTAINER_ALLOCATED)
.setContainer(container);
dispatcher.getEventHandler().handle(event);
LOG.info("[COMPONENT {}]: {} outstanding container requests.",
comp.getName(),
amRMClient.getMatchingRequests(container.getAllocationRequestId()).size());
// remove the corresponding request
Collection<AMRMClient.ContainerRequest> collection = amRMClient
.getMatchingRequests(container.getAllocationRequestId());
if (collection.iterator().hasNext()) {
AMRMClient.ContainerRequest request = collection.iterator().next();
amRMClient.removeContainerRequest(request);
}
}
}
@Override
public void onContainersCompleted(List<ContainerStatus> statuses) {
for (ContainerStatus status : statuses) {
ContainerId containerId = status.getContainerId();
ComponentInstance instance = liveInstances.get(status.getContainerId());
if (instance == null) {
LOG.error(
"Container {} Completed. No component instance exists. exitStatus={}. diagnostics={} ",
containerId, status.getExitStatus(), status.getDiagnostics());
return;
}
ComponentEvent event =
new ComponentEvent(instance.getCompName(), CONTAINER_COMPLETED)
.setStatus(status).setInstance(instance);
dispatcher.getEventHandler().handle(event);
}
}
@Override
public void onContainersUpdated(List<UpdatedContainer> containers) {
}
@Override public void onShutdownRequest() {
//Was used for non-work-preserving restart in YARN, should be deprecated.
}
@Override public void onNodesUpdated(List<NodeReport> updatedNodes) {
StringBuilder str = new StringBuilder();
str.append("Nodes updated info: ").append(System.lineSeparator());
for (NodeReport report : updatedNodes) {
str.append(report.getNodeId()).append(", state = ")
.append(report.getNodeState()).append(", healthDiagnostics = ")
.append(report.getHealthReport()).append(System.lineSeparator());
}
LOG.warn(str.toString());
}
@Override public float getProgress() {
// get running containers over desired containers
long total = 0;
for (org.apache.slider.api.resource.Component component : app
.getComponents()) {
total += component.getNumberOfContainers();
}
// Probably due to user flexed down to 0
if (total == 0) {
return 100;
}
return Math.max((float) liveInstances.size() / total * 100, 100);
}
@Override public void onError(Throwable e) {
LOG.error("Error in AMRMClient callback handler ", e);
}
}
private class NMClientCallback extends NMClientAsync.AbstractCallbackHandler {
@Override public void onContainerStarted(ContainerId containerId,
Map<String, ByteBuffer> allServiceResponse) {
ComponentInstance instance = liveInstances.get(containerId);
if (instance == null) {
LOG.error("No component instance exists for " + containerId);
return;
}
ComponentEvent event =
new ComponentEvent(instance.getCompName(), CONTAINER_STARTED)
.setInstance(instance);
dispatcher.getEventHandler().handle(event);
}
@Override public void onContainerStatusReceived(ContainerId containerId,
ContainerStatus containerStatus) {
}
@Override public void onContainerStopped(ContainerId containerId) {
}
@Override
public void onStartContainerError(ContainerId containerId, Throwable t) {
ComponentInstance instance = liveInstances.get(containerId);
if (instance == null) {
LOG.error("No component instance exists for " + containerId);
return;
}
amRMClient.releaseAssignedContainer(containerId);
// After container released, it'll get CONTAINER_COMPLETED event from RM
// automatically which will trigger stopping COMPONENT INSTANCE
}
@Override public void onContainerResourceIncreased(ContainerId containerId,
Resource resource) {
}
@Override public void onGetContainerStatusError(ContainerId containerId,
Throwable t) {
}
@Override
public void onIncreaseContainerResourceError(ContainerId containerId,
Throwable t) {
}
@Override
public void onStopContainerError(ContainerId containerId, Throwable t) {
}
}
public ServiceMetrics getServiceMetrics() {
return serviceMetrics;
}
public AMRMClientAsync<AMRMClient.ContainerRequest> getAmRMClient() {
return amRMClient;
}
public NMClientAsync getNmClient() {
return nmClient;
}
public void addLiveCompInstance(ContainerId containerId,
ComponentInstance instance) {
liveInstances.put(containerId, instance);
}
public void removeLiveCompInstance(ContainerId containerId) {
liveInstances.remove(containerId);
}
public AsyncDispatcher getCompInstanceDispatcher() {
return compInstanceDispatcher;
}
public YarnRegistryViewForProviders getYarnRegistryOperations() {
return yarnRegistryOperations;
}
public ServiceTimelinePublisher getServiceTimelinePublisher() {
return serviceTimelinePublisher;
}
public Map<ContainerId, ComponentInstance> getLiveInstances() {
return liveInstances;
}
public ContainerLaunchService getContainerLaunchService() {
return containerLaunchService;
}
public ServiceContext getContext() {
return context;
}
public Map<String, Component> getAllComponents() {
return componentsByName;
}
public Application getApp() {
return app;
}
public AsyncDispatcher getDispatcher() {
return dispatcher;
}
public BoundedAppender getDiagnostics() {
return diagnostics;
}
}

View File

@ -0,0 +1,49 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.client;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.client.ServerProxy;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.service.conf.YarnServiceConfKeys;
import java.net.InetSocketAddress;
public class ClientAMProxy extends ServerProxy{
public static <T> T createProxy(final Configuration conf,
final Class<T> protocol, final UserGroupInformation ugi,
final YarnRPC rpc, final InetSocketAddress serverAddress) {
RetryPolicy retryPolicy =
createRetryPolicy(conf, YarnServiceConfKeys.CLIENT_AM_RETRY_MAX_WAIT_MS,
15 * 60 * 1000, YarnServiceConfKeys.CLIENT_AM_RETRY_MAX_INTERVAL_MS,
2 * 1000);
Configuration confClone = new Configuration(conf);
confClone.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
confClone.setInt(CommonConfigurationKeysPublic.
IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY, 0);
return createRetriableProxy(confClone, protocol, ugi, rpc, serverAddress,
retryPolicy);
}
}

View File

@ -0,0 +1,98 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.client;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.slider.api.resource.Application;
import org.apache.hadoop.yarn.service.client.params.ClientArgs;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.*;
public class ServiceCLI {
private static final Logger LOG =
LoggerFactory.getLogger(ServiceClient.class);
protected ServiceClient client;
public int exec(ClientArgs args) throws Throwable {
if (StringUtils.isEmpty(args.getAction())) {
System.out.println(args.usage());
return -1;
}
switch (args.getAction()) {
case ACTION_BUILD: // Upload app json onto hdfs
client.actionBuild(args.getActionBuildArgs());
break;
case ACTION_START: // start the app with the pre-uploaded app json on hdfs
client.actionStart(args.getClusterName());
break;
case ACTION_CREATE: // create == build + start
client.actionCreate(args.getActionCreateArgs());
break;
case ACTION_STATUS:
Application app = client.getStatus(args.getClusterName());
System.out.println(app);
break;
case ACTION_FLEX:
client.actionFlexByCLI(args);
break;
case ACTION_STOP:
client.actionStop(args.getClusterName());
break;
case ACTION_DESTROY: // Destroy can happen only if app is already stopped
client.actionDestroy(args.getClusterName());
break;
case ACTION_DEPENDENCY: // upload dependency jars
client.actionDependency(args.getActionDependencyArgs());
break;
case ACTION_UPDATE:
client.updateLifetime(args.getClusterName(),
args.getActionUpdateArgs().lifetime);
break;
case ACTION_HELP:
LOG.info(args.usage());
break;
default:
LOG.info("NOT IMPLEMENTED: " + args.getAction());
LOG.info(args.usage());
return -1;
}
return 0;
}
public ServiceCLI() {
createServiceClient();
}
protected void createServiceClient() {
client = new ServiceClient();
client.init(new YarnConfiguration());
client.start();
}
public static void main(String[] args) throws Throwable {
ClientArgs clientArgs = new ClientArgs(args);
clientArgs.parse();
ServiceCLI cli = new ServiceCLI();
int res = cli.exec(clientArgs);
System.exit(res);
}
}

View File

@ -0,0 +1,836 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.client;
import org.apache.commons.lang.StringUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.RetryNTimes;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.registry.client.api.RegistryConstants;
import org.apache.hadoop.registry.client.api.RegistryOperations;
import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
import org.apache.hadoop.registry.client.binding.RegistryUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.client.api.YarnClientApplication;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.ComponentCountProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto;
import org.apache.hadoop.yarn.service.ClientAMProtocol;
import org.apache.hadoop.yarn.service.ServiceMaster;
import org.apache.hadoop.yarn.service.client.params.ActionDependencyArgs;
import org.apache.hadoop.yarn.service.client.params.ActionFlexArgs;
import org.apache.hadoop.yarn.service.client.params.Arguments;
import org.apache.hadoop.yarn.service.client.params.ClientArgs;
import org.apache.hadoop.yarn.service.client.params.CommonArgs;
import org.apache.hadoop.yarn.service.conf.SliderExitCodes;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.hadoop.yarn.service.provider.ProviderUtils;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.Times;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.Component;
import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.exceptions.BadClusterStateException;
import org.apache.slider.core.exceptions.BadConfigException;
import org.apache.slider.core.exceptions.SliderException;
import org.apache.slider.core.exceptions.UsageException;
import org.apache.slider.core.launch.ClasspathConstructor;
import org.apache.slider.core.launch.JavaCommandLineBuilder;
import org.apache.slider.core.registry.SliderRegistryUtils;
import org.apache.slider.core.zk.ZKIntegration;
import org.apache.slider.core.zk.ZookeeperUtils;
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.text.MessageFormat;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import static org.apache.hadoop.yarn.api.records.YarnApplicationState.*;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_CREATE;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_FLEX;
import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG;
import static org.apache.slider.common.tools.SliderUtils.*;
public class ServiceClient extends CompositeService
implements SliderExitCodes, SliderKeys {
private static final Logger LOG =
LoggerFactory.getLogger(ServiceClient.class);
private SliderFileSystem fs;
private YarnClient yarnClient;
// Avoid looking up applicationId from fs all the time.
private Map<String, ApplicationId> cachedAppIds = new ConcurrentHashMap<>();
private RegistryOperations registryClient;
private CuratorFramework curatorClient;
private YarnRPC rpc;
private static EnumSet<YarnApplicationState> terminatedStates =
EnumSet.of(FINISHED, FAILED, KILLED);
private static EnumSet<YarnApplicationState> liveStates =
EnumSet.of(NEW, NEW_SAVING, SUBMITTED, RUNNING);
public ServiceClient() {
super(ServiceClient.class.getName());
}
@Override protected void serviceInit(Configuration configuration)
throws Exception {
fs = new SliderFileSystem(configuration);
yarnClient = YarnClient.createYarnClient();
rpc = YarnRPC.create(configuration);
addService(yarnClient);
super.serviceInit(configuration);
}
@Override
protected void serviceStop() throws Exception {
if (registryClient != null) {
registryClient.stop();
}
super.serviceStop();
}
private Application loadAppJsonFromLocalFS(
AbstractClusterBuildingActionArgs args) throws IOException {
File file = args.getAppDef();
Path filePath = new Path(file.getAbsolutePath());
LOG.info("Loading app json from: " + filePath);
Application application = ServiceApiUtil.jsonSerDeser
.load(FileSystem.getLocal(getConfig()), filePath);
if (args.lifetime > 0) {
application.setLifetime(args.lifetime);
}
application.setName(args.getClusterName());
return application;
}
public int actionBuild(AbstractClusterBuildingActionArgs args)
throws IOException, YarnException {
return actionBuild(loadAppJsonFromLocalFS(args));
}
public int actionBuild(Application application)
throws YarnException, IOException {
Path appDir = checkAppNotExistOnHdfs(application);
ServiceApiUtil.validateAndResolveApplication(application, fs, getConfig());
createDirAndPersistApp(appDir, application);
return EXIT_SUCCESS;
}
public int actionCreate(AbstractClusterBuildingActionArgs args)
throws IOException, YarnException {
actionCreate(loadAppJsonFromLocalFS(args));
return EXIT_SUCCESS;
}
public ApplicationId actionCreate(Application application)
throws IOException, YarnException {
String appName = application.getName();
validateClusterName(appName);
ServiceApiUtil.validateAndResolveApplication(application, fs, getConfig());
verifyNoLiveAppInRM(appName, "create");
Path appDir = checkAppNotExistOnHdfs(application);
// Write the definition first and then submit - AM will read the definition
createDirAndPersistApp(appDir, application);
ApplicationId appId = submitApp(application);
cachedAppIds.put(appName, appId);
application.setId(appId.toString());
// update app definition with appId
persistAppDef(appDir, application);
return appId;
}
// Called by ServiceCLI
protected int actionFlexByCLI(ClientArgs args)
throws YarnException, IOException {
ActionFlexArgs flexArgs = args.getActionFlexArgs();
Map<String, Long> componentCounts =
new HashMap<>(flexArgs.getComponentMap().size());
Application persistedApp =
ServiceApiUtil.loadApplication(fs, flexArgs.getClusterName());
if (!StringUtils.isEmpty(persistedApp.getId())) {
cachedAppIds.put(persistedApp.getName(),
ApplicationId.fromString(persistedApp.getId()));
}
for (Map.Entry<String, String> entry : flexArgs.getComponentMap()
.entrySet()) {
String compName = entry.getKey();
ServiceApiUtil.validateCompName(compName);
Component component = persistedApp.getComponent(compName);
if (component == null) {
throw new IllegalArgumentException(entry.getKey() + " does not exist !");
}
long numberOfContainers =
parseNumberOfContainers(component, entry.getValue());
componentCounts.put(compName, numberOfContainers);
}
// throw usage exception if no changes proposed
if (componentCounts.size() == 0) {
actionHelp(ACTION_FLEX, args);
}
flexComponents(args.getClusterName(), componentCounts, persistedApp);
return EXIT_SUCCESS;
}
// Parse the number of containers requested by user, e.g.
// +5 means add 5 additional containers
// -5 means reduce 5 containers, if it goes to negative, sets it to 0
// 5 means sets it to 5 containers.
private long parseNumberOfContainers(Component component, String newNumber) {
long orig = component.getNumberOfContainers();
if (newNumber.startsWith("+")) {
return orig + Long.parseLong(newNumber.substring(1));
} else if (newNumber.startsWith("-")) {
long ret = orig - Long.parseLong(newNumber.substring(1));
if (ret < 0) {
LOG.warn(MessageFormat.format(
"[COMPONENT {}]: component count goes to negative ({}{} = {}), reset it to 0.",
component.getName(), orig, newNumber, ret));
ret = 0;
}
return ret;
} else {
return Long.parseLong(newNumber);
}
}
// Called by Rest Service
public Map<String, Long> flexByRestService(String appName,
Map<String, Long> componentCounts) throws YarnException, IOException {
// load app definition
Application persistedApp = ServiceApiUtil.loadApplication(fs, appName);
cachedAppIds.put(persistedApp.getName(),
ApplicationId.fromString(persistedApp.getId()));
return flexComponents(appName, componentCounts, persistedApp);
}
private Map<String, Long> flexComponents(String appName,
Map<String, Long> componentCounts, Application persistedApp)
throws YarnException, IOException {
validateClusterName(appName);
Map<String, Long> original = new HashMap<>(componentCounts.size());
ComponentCountProto.Builder countBuilder = ComponentCountProto.newBuilder();
FlexComponentsRequestProto.Builder requestBuilder =
FlexComponentsRequestProto.newBuilder();
for (Component persistedComp : persistedApp.getComponents()) {
String name = persistedComp.getName();
if (componentCounts.containsKey(persistedComp.getName())) {
original.put(name, persistedComp.getNumberOfContainers());
persistedComp.setNumberOfContainers(componentCounts.get(name));
// build the request
countBuilder.setName(persistedComp.getName())
.setNumberOfContainers(persistedComp.getNumberOfContainers());
requestBuilder.addComponents(countBuilder.build());
}
}
if (original.size() < componentCounts.size()) {
componentCounts.keySet().removeAll(original.keySet());
throw new YarnException("Components " + componentCounts.keySet()
+ " do not exist in app definition.");
}
ServiceApiUtil.jsonSerDeser
.save(fs.getFileSystem(), ServiceApiUtil.getAppJsonPath(fs, appName),
persistedApp, true);
ClientAMProtocol proxy = connectToAM(appName);
proxy.flexComponents(requestBuilder.build());
for (Map.Entry<String, Long> entry : original.entrySet()) {
LOG.info("[COMPONENT {}]: number of containers changed from {} to {}",
entry.getKey(), entry.getValue(),
componentCounts.get(entry.getKey()));
}
return original;
}
public int actionStop(String appName) throws YarnException, IOException {
validateClusterName(appName);
getAppIdFromPersistedApp(appName);
ApplicationId currentAppId = cachedAppIds.get(appName);
ApplicationReport report = yarnClient.getApplicationReport(currentAppId);
if (terminatedStates.contains(report.getYarnApplicationState())) {
LOG.info("Application {} is already in a terminated state {}", appName,
report.getYarnApplicationState());
return EXIT_SUCCESS;
}
LOG.info("Stopping application {}, with appId = {}", appName, currentAppId);
try {
// try to stop the app gracefully.
ClientAMProtocol proxy = connectToAM(appName);
StopRequestProto request = StopRequestProto.newBuilder().build();
proxy.stop(request);
LOG.info("Application " + appName + " is being gracefully stopped...");
// Wait until the app is killed.
long startTime = System.currentTimeMillis();
int pollCount = 0;
while (true) {
Thread.sleep(1000);
report = yarnClient.getApplicationReport(currentAppId);
if (terminatedStates.contains(report.getYarnApplicationState())) {
LOG.info("Application " + appName + " is stopped.");
break;
}
// Forcefully kill after 10 seconds.
if ((System.currentTimeMillis() - startTime) > 10000) {
LOG.info("Stop operation timeout stopping, forcefully kill the app "
+ appName);
yarnClient.killApplication(currentAppId,
"Forcefully kill the app by user");
break;
}
if (++pollCount % 10 == 0) {
LOG.info("Waiting for application " + appName + " to be stopped.");
}
}
} catch (IOException | YarnException | InterruptedException e) {
LOG.info("Failed to stop " + appName
+ " gracefully, forcefully kill the app.");
yarnClient.killApplication(currentAppId, "Forcefully kill the app");
}
return EXIT_SUCCESS;
}
public int actionDestroy(String appName) throws Exception {
validateClusterName(appName);
verifyNoLiveAppInRM(appName, "Destroy");
Path appDir = fs.buildClusterDirPath(appName);
FileSystem fileSystem = fs.getFileSystem();
// remove from the appId cache
cachedAppIds.remove(appName);
if (fileSystem.exists(appDir)) {
if (fileSystem.delete(appDir, true)) {
LOG.info("Successfully deleted application dir for " + appName + ": "
+ appDir);
} else {
String message =
"Failed to delete application + " + appName + " at: " + appDir;
LOG.info(message);
throw new YarnException(message);
}
}
deleteZKNode(appName);
String registryPath = SliderRegistryUtils.registryPathForInstance(appName);
try {
getRegistryClient().delete(registryPath, true);
} catch (IOException e) {
LOG.warn("Error deleting registry entry {}", registryPath, e);
}
LOG.info("Destroyed cluster {}", appName);
return EXIT_SUCCESS;
}
private synchronized RegistryOperations getRegistryClient()
throws SliderException, IOException {
if (registryClient == null) {
registryClient =
RegistryOperationsFactory.createInstance("ServiceClient", getConfig());
registryClient.init(getConfig());
registryClient.start();
}
return registryClient;
}
private void deleteZKNode(String clusterName) throws Exception {
CuratorFramework curatorFramework = getCuratorClient();
String user = RegistryUtils.currentUser();
String zkPath = ZKIntegration.mkClusterPath(user, clusterName);
if (curatorFramework.checkExists().forPath(zkPath) != null) {
curatorFramework.delete().deletingChildrenIfNeeded().forPath(zkPath);
LOG.info("Deleted zookeeper path: " + zkPath);
}
}
private synchronized CuratorFramework getCuratorClient()
throws BadConfigException {
String registryQuorum =
getConfig().get(RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
// though if neither is set: trouble
if (SliderUtils.isUnset(registryQuorum)) {
throw new BadConfigException(
"No Zookeeper quorum provided in the" + " configuration property "
+ RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
}
ZookeeperUtils.splitToHostsAndPortsStrictly(registryQuorum);
if (curatorClient == null) {
curatorClient =
CuratorFrameworkFactory.builder().connectString(registryQuorum)
.sessionTimeoutMs(10000).retryPolicy(new RetryNTimes(10, 2000))
.build();
curatorClient.start();
}
return curatorClient;
}
private int actionHelp(String actionName, CommonArgs args)
throws YarnException, IOException {
throw new UsageException(CommonArgs.usage(args, actionName));
}
private void verifyNoLiveAppInRM(String appname, String action)
throws IOException, YarnException {
Set<String> types = new HashSet<>(1);
types.add(SliderKeys.APP_TYPE);
Set<String> tags = null;
if (appname != null) {
tags = Collections.singleton(SliderUtils.createNameTag(appname));
}
GetApplicationsRequest request = GetApplicationsRequest.newInstance();
request.setApplicationTypes(types);
request.setApplicationTags(tags);
request.setApplicationStates(liveStates);
List<ApplicationReport> reports = yarnClient.getApplications(request);
if (!reports.isEmpty()) {
throw new YarnException(
"Failed to " + action + " application, as " + appname
+ " already exists.");
}
}
private ApplicationId submitApp(Application app)
throws IOException, YarnException {
String appName = app.getName();
Configuration conf = getConfig();
Path appRootDir = fs.buildClusterDirPath(app.getName());
YarnClientApplication yarnApp = yarnClient.createApplication();
ApplicationSubmissionContext submissionContext =
yarnApp.getApplicationSubmissionContext();
ServiceApiUtil.validateCompResourceSize(
yarnApp.getNewApplicationResponse().getMaximumResourceCapability(),
app);
submissionContext.setKeepContainersAcrossApplicationAttempts(true);
if (app.getLifetime() > 0) {
Map<ApplicationTimeoutType, Long> appTimeout = new HashMap<>();
appTimeout.put(ApplicationTimeoutType.LIFETIME, app.getLifetime());
submissionContext.setApplicationTimeouts(appTimeout);
}
submissionContext.setMaxAppAttempts(conf.getInt(KEY_AM_RESTART_LIMIT, 2));
Map<String, LocalResource> localResources = new HashMap<>();
// copy local slideram-log4j.properties to hdfs and add to localResources
boolean hasSliderAMLog4j =
addAMLog4jResource(appName, conf, localResources);
// copy jars to hdfs and add to localResources
addJarResource(appName, localResources);
// add keytab if in secure env
addKeytabResourceIfSecure(fs, localResources, conf, appName);
if (LOG.isDebugEnabled()) {
printLocalResources(localResources);
}
Map<String, String> env = addAMEnv(conf);
// create AM CLI
String cmdStr =
buildCommandLine(appName, conf, appRootDir, hasSliderAMLog4j);
submissionContext.setResource(Resource.newInstance(
conf.getLong(KEY_AM_RESOURCE_MEM, DEFAULT_KEY_AM_RESOURCE_MEM), 1));
submissionContext.setQueue(conf.get(KEY_YARN_QUEUE, app.getQueue()));
submissionContext.setApplicationName(appName);
submissionContext.setApplicationType(SliderKeys.APP_TYPE);
Set<String> appTags =
AbstractClientProvider.createApplicationTags(appName, null, null);
if (!appTags.isEmpty()) {
submissionContext.setApplicationTags(appTags);
}
ContainerLaunchContext amLaunchContext =
Records.newRecord(ContainerLaunchContext.class);
amLaunchContext.setCommands(Collections.singletonList(cmdStr));
amLaunchContext.setEnvironment(env);
amLaunchContext.setLocalResources(localResources);
submissionContext.setAMContainerSpec(amLaunchContext);
yarnClient.submitApplication(submissionContext);
return submissionContext.getApplicationId();
}
private void printLocalResources(Map<String, LocalResource> map) {
LOG.debug("Added LocalResource for localization: ");
StringBuilder builder = new StringBuilder();
for (Map.Entry<String, LocalResource> entry : map.entrySet()) {
builder.append(entry.getKey()).append(" -> ")
.append(entry.getValue().getResource().getFile())
.append(System.lineSeparator());
}
LOG.debug(builder.toString());
}
private String buildCommandLine(String appName, Configuration conf,
Path appRootDir, boolean hasSliderAMLog4j) throws BadConfigException {
JavaCommandLineBuilder CLI = new JavaCommandLineBuilder();
CLI.forceIPv4().headless();
//TODO CLI.setJVMHeap
//TODO CLI.addJVMOPTS
if (hasSliderAMLog4j) {
CLI.sysprop(SYSPROP_LOG4J_CONFIGURATION, LOG4J_SERVER_PROP_FILENAME);
CLI.sysprop(SYSPROP_LOG_DIR, ApplicationConstants.LOG_DIR_EXPANSION_VAR);
}
CLI.add(ServiceMaster.class.getCanonicalName());
CLI.add(ACTION_CREATE, appName);
//TODO debugAM CLI.add(Arguments.ARG_DEBUG)
CLI.add(Arguments.ARG_CLUSTER_URI, new Path(appRootDir, appName + ".json"));
// pass the registry binding
CLI.addConfOptionToCLI(conf, RegistryConstants.KEY_REGISTRY_ZK_ROOT,
RegistryConstants.DEFAULT_ZK_REGISTRY_ROOT);
CLI.addMandatoryConfOption(conf, RegistryConstants.KEY_REGISTRY_ZK_QUORUM);
// write out the path output
CLI.addOutAndErrFiles(STDOUT_AM, STDERR_AM);
String cmdStr = CLI.build();
LOG.info("AM launch command: {}", cmdStr);
return cmdStr;
}
private Map<String, String> addAMEnv(Configuration conf) throws IOException {
Map<String, String> env = new HashMap<>();
ClasspathConstructor classpath =
buildClasspath(SliderKeys.SUBMITTED_CONF_DIR, "lib", fs, getConfig()
.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false));
env.put("CLASSPATH", classpath.buildClasspath());
env.put("LANG", "en_US.UTF-8");
env.put("LC_ALL", "en_US.UTF-8");
env.put("LANGUAGE", "en_US.UTF-8");
String jaas = System.getenv(HADOOP_JAAS_DEBUG);
if (jaas != null) {
env.put(HADOOP_JAAS_DEBUG, jaas);
}
if (!UserGroupInformation.isSecurityEnabled()) {
String userName = UserGroupInformation.getCurrentUser().getUserName();
LOG.info("Run as user " + userName);
// HADOOP_USER_NAME env is used by UserGroupInformation when log in
// This env makes AM run as this user
env.put("HADOOP_USER_NAME", userName);
}
LOG.info("AM env: \n{}", stringifyMap(env));
return env;
}
protected Path addJarResource(String appName,
Map<String, LocalResource> localResources)
throws IOException, SliderException {
Path libPath = fs.buildClusterDirPath(appName);
ProviderUtils
.addProviderJar(localResources, ServiceMaster.class, SLIDER_JAR, fs,
libPath, "lib", false);
Path dependencyLibTarGzip = fs.getDependencyTarGzip();
if (fs.isFile(dependencyLibTarGzip)) {
LOG.info("Loading lib tar from " + fs.getFileSystem().getScheme() + ": "
+ dependencyLibTarGzip);
SliderUtils.putAmTarGzipAndUpdate(localResources, fs);
} else {
String[] libs = SliderUtils.getLibDirs();
for (String libDirProp : libs) {
ProviderUtils.addAllDependencyJars(localResources, fs, libPath, "lib",
libDirProp);
}
}
return libPath;
}
private boolean addAMLog4jResource(String appName, Configuration conf,
Map<String, LocalResource> localResources)
throws IOException, BadClusterStateException {
boolean hasSliderAMLog4j = false;
String hadoopConfDir =
System.getenv(ApplicationConstants.Environment.HADOOP_CONF_DIR.name());
if (hadoopConfDir != null) {
File localFile =
new File(hadoopConfDir, SliderKeys.LOG4J_SERVER_PROP_FILENAME);
if (localFile.exists()) {
Path localFilePath = createLocalPath(localFile);
Path appDirPath = fs.buildClusterDirPath(appName);
Path remoteConfPath =
new Path(appDirPath, SliderKeys.SUBMITTED_CONF_DIR);
Path remoteFilePath =
new Path(remoteConfPath, SliderKeys.LOG4J_SERVER_PROP_FILENAME);
copy(conf, localFilePath, remoteFilePath);
LocalResource localResource =
fs.createAmResource(remoteConfPath, LocalResourceType.FILE);
localResources.put(localFilePath.getName(), localResource);
hasSliderAMLog4j = true;
}
}
return hasSliderAMLog4j;
}
public int actionStart(String appName) throws YarnException, IOException {
validateClusterName(appName);
Path appDir = checkAppExistOnHdfs(appName);
Application application = ServiceApiUtil.loadApplication(fs, appName);
ServiceApiUtil.validateAndResolveApplication(application, fs, getConfig());
// see if it is actually running and bail out;
verifyNoLiveAppInRM(appName, "thaw");
ApplicationId appId = submitApp(application);
application.setId(appId.toString());
// write app definition on to hdfs
createDirAndPersistApp(appDir, application);
return 0;
}
private Path checkAppNotExistOnHdfs(Application application)
throws IOException, SliderException {
Path appDir = fs.buildClusterDirPath(application.getName());
fs.verifyDirectoryNonexistent(
new Path(appDir, application.getName() + ".json"));
return appDir;
}
private Path checkAppExistOnHdfs(String appName)
throws IOException, SliderException {
Path appDir = fs.buildClusterDirPath(appName);
fs.verifyPathExists(new Path(appDir, appName + ".json"));
return appDir;
}
private void createDirAndPersistApp(Path appDir, Application application)
throws IOException, SliderException {
FsPermission appDirPermission = new FsPermission("750");
fs.createWithPermissions(appDir, appDirPermission);
persistAppDef(appDir, application);
}
private void persistAppDef(Path appDir, Application application)
throws IOException {
Path appJson = new Path(appDir, application.getName() + ".json");
ServiceApiUtil.jsonSerDeser
.save(fs.getFileSystem(), appJson, application, true);
LOG.info(
"Persisted application " + application.getName() + " at " + appJson);
}
private void addKeytabResourceIfSecure(SliderFileSystem fileSystem,
Map<String, LocalResource> localResource, Configuration conf,
String appName) throws IOException, BadConfigException {
if (!UserGroupInformation.isSecurityEnabled()) {
return;
}
String keytabPreInstalledOnHost =
conf.get(SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
if (StringUtils.isEmpty(keytabPreInstalledOnHost)) {
String amKeytabName =
conf.get(SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
String keytabDir = conf.get(SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR);
Path keytabPath =
fileSystem.buildKeytabPath(keytabDir, amKeytabName, appName);
if (fileSystem.getFileSystem().exists(keytabPath)) {
LocalResource keytabRes =
fileSystem.createAmResource(keytabPath, LocalResourceType.FILE);
localResource
.put(SliderKeys.KEYTAB_DIR + "/" + amKeytabName, keytabRes);
LOG.info("Adding AM keytab on hdfs: " + keytabPath);
} else {
LOG.warn("No keytab file was found at {}.", keytabPath);
if (conf.getBoolean(KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) {
throw new BadConfigException("No keytab file was found at %s.",
keytabPath);
} else {
LOG.warn("The AM will be "
+ "started without a kerberos authenticated identity. "
+ "The application is therefore not guaranteed to remain "
+ "operational beyond 24 hours.");
}
}
}
}
public String updateLifetime(String appName, long lifetime)
throws YarnException, IOException {
getAppIdFromPersistedApp(appName);
ApplicationId currentAppId = cachedAppIds.get(appName);
ApplicationReport report = yarnClient.getApplicationReport(currentAppId);
if (report == null) {
throw new YarnException("Application not found for " + appName);
}
ApplicationId appId = report.getApplicationId();
LOG.info("Updating lifetime of an application: appName = " + appName
+ ", appId = " + appId + ", lifetime = " + lifetime);
Map<ApplicationTimeoutType, String> map = new HashMap<>();
String newTimeout =
Times.formatISO8601(System.currentTimeMillis() + lifetime * 1000);
map.put(ApplicationTimeoutType.LIFETIME, newTimeout);
UpdateApplicationTimeoutsRequest request =
UpdateApplicationTimeoutsRequest.newInstance(appId, map);
yarnClient.updateApplicationTimeouts(request);
LOG.info(
"Successfully updated lifetime for an application: appName = " + appName
+ ", appId = " + appId + ". New expiry time in ISO8601 format is "
+ newTimeout);
return newTimeout;
}
public Application getStatus(String appName)
throws IOException, YarnException {
ClientAMProtocol proxy = connectToAM(appName);
GetStatusResponseProto response =
proxy.getStatus(GetStatusRequestProto.newBuilder().build());
return ServiceApiUtil.jsonSerDeser.fromJson(response.getStatus());
}
public YarnClient getYarnClient() {
return this.yarnClient;
}
public int actionDependency(ActionDependencyArgs args)
throws IOException, YarnException {
String currentUser = RegistryUtils.currentUser();
LOG.info("Running command as user {}", currentUser);
Path dependencyLibTarGzip = fs.getDependencyTarGzip();
// Check if dependency has already been uploaded, in which case log
// appropriately and exit success (unless overwrite has been requested)
if (fs.isFile(dependencyLibTarGzip) && !args.overwrite) {
System.out.println(String.format(
"Dependency libs are already uploaded to %s. Use %s "
+ "if you want to re-upload", dependencyLibTarGzip.toUri(),
Arguments.ARG_OVERWRITE));
return EXIT_SUCCESS;
}
String[] libDirs = SliderUtils.getLibDirs();
if (libDirs.length > 0) {
File tempLibTarGzipFile = File.createTempFile(
SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_NAME + "_",
SliderKeys.SLIDER_DEPENDENCY_TAR_GZ_FILE_EXT);
// copy all jars
tarGzipFolder(libDirs, tempLibTarGzipFile, createJarFilter());
LOG.info("Uploading dependency for AM (version {}) from {} to {}",
VersionInfo.getBuildVersion(), tempLibTarGzipFile.toURI(),
dependencyLibTarGzip.toUri());
fs.copyLocalFileToHdfs(tempLibTarGzipFile, dependencyLibTarGzip,
new FsPermission(SliderKeys.SLIDER_DEPENDENCY_DIR_PERMISSIONS));
return EXIT_SUCCESS;
} else {
return EXIT_FALSE;
}
}
protected ClientAMProtocol connectToAM(String appName)
throws IOException, YarnException {
ApplicationId currentAppId = getAppIdFromPersistedApp(appName);
// Wait until app becomes running.
long startTime = System.currentTimeMillis();
int pollCount = 0;
ApplicationReport appReport = null;
while (true) {
appReport = yarnClient.getApplicationReport(currentAppId);
YarnApplicationState state = appReport.getYarnApplicationState();
if (state == RUNNING) {
break;
}
if (terminatedStates.contains(state)) {
throw new YarnException(
"Failed to getStatus " + currentAppId + ": " + appReport
.getDiagnostics());
}
long elapsedMillis = System.currentTimeMillis() - startTime;
// if over 5 min, quit
if (elapsedMillis >= 300000) {
throw new YarnException(
"Timed out while waiting for application " + currentAppId
+ " to be running");
}
if (++pollCount % 10 == 0) {
LOG.info(
"Waiting for application {} to be running, current state is {}",
currentAppId, state);
}
try {
Thread.sleep(3000);
} catch (InterruptedException ie) {
String msg =
"Interrupted while waiting for application " + currentAppId
+ " to be running.";
throw new YarnException(msg, ie);
}
}
// Make the connection
InetSocketAddress address = NetUtils
.createSocketAddrForHost(appReport.getHost(), appReport.getRpcPort());
return ClientAMProxy.createProxy(getConfig(), ClientAMProtocol.class,
UserGroupInformation.getCurrentUser(), rpc, address);
}
private synchronized ApplicationId getAppIdFromPersistedApp(String appName)
throws IOException, YarnException {
if (cachedAppIds.containsKey(appName)) {
return cachedAppIds.get(appName);
}
Application persistedApp = ServiceApiUtil.loadApplication(fs, appName);
if (persistedApp == null) {
throw new YarnException("Application " + appName
+ " doesn't exist on hdfs. Please check if the app exists in RM");
}
ApplicationId currentAppId = ApplicationId.fromString(persistedApp.getId());
cachedAppIds.put(appName, currentAppId);
return currentAppId;
}
}

View File

@ -16,10 +16,11 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.Parameter;
import org.apache.hadoop.fs.Path;
import org.apache.slider.common.params.PathArgumentConverter;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.ErrorStrings;
import org.apache.slider.core.exceptions.UsageException;

View File

@ -15,11 +15,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
@Parameters(commandNames = {SliderActions.ACTION_BUILD},
@Parameters(commandNames = { SliderActions.ACTION_BUILD},
commandDescription = SliderActions.DESCRIBE_ACTION_BUILD)
public class ActionBuildArgs extends AbstractClusterBuildingActionArgs {

View File

@ -16,11 +16,13 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
@Parameters(commandNames = {SliderActions.ACTION_CREATE},
@Parameters(commandNames = { SliderActions.ACTION_CREATE},
commandDescription = SliderActions.DESCRIBE_ACTION_CREATE)
public class ActionCreateArgs extends AbstractClusterBuildingActionArgs {

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.UsageException;

View File

@ -16,12 +16,12 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
@Parameters(commandNames = {SliderActions.ACTION_DESTROY},
@Parameters(commandNames = { SliderActions.ACTION_DESTROY},
commandDescription = SliderActions.DESCRIBE_ACTION_DESTROY)
public class ActionDestroyArgs extends AbstractActionArgs {

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.Parameters;
import com.beust.jcommander.ParametersDelegate;
@ -25,7 +25,7 @@
import java.util.List;
import java.util.Map;
@Parameters(commandNames = {SliderActions.ACTION_FLEX},
@Parameters(commandNames = { SliderActions.ACTION_FLEX},
commandDescription = SliderActions.DESCRIBE_ACTION_FLEX)
public class ActionFlexArgs extends AbstractActionArgs {

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -25,7 +25,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
/**
* Here are all the arguments that may be parsed by the client or server

View File

@ -16,11 +16,33 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.slider.common.SliderXmlConfKeys;
import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys;
import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
import org.apache.slider.common.params.ActionAMSuicideArgs;
import org.apache.slider.common.params.ActionClientArgs;
import org.apache.slider.common.params.ActionDiagnosticArgs;
import org.apache.slider.common.params.ActionExistsArgs;
import org.apache.slider.common.params.ActionFreezeArgs;
import org.apache.slider.common.params.ActionHelpArgs;
import org.apache.slider.common.params.ActionKDiagArgs;
import org.apache.slider.common.params.ActionKeytabArgs;
import org.apache.slider.common.params.ActionKillContainerArgs;
import org.apache.slider.common.params.ActionListArgs;
import org.apache.slider.common.params.ActionLookupArgs;
import org.apache.slider.common.params.ActionNodesArgs;
import org.apache.slider.common.params.ActionRegistryArgs;
import org.apache.slider.common.params.ActionResolveArgs;
import org.apache.slider.common.params.ActionResourceArgs;
import org.apache.slider.common.params.ActionStatusArgs;
import org.apache.slider.common.params.ActionThawArgs;
import org.apache.slider.common.params.ActionTokensArgs;
import org.apache.slider.common.params.ActionUpdateArgs;
import org.apache.slider.common.params.ActionUpgradeArgs;
import org.apache.slider.common.params.ActionVersionArgs;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.ErrorStrings;

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
@ -243,15 +243,9 @@ public void validate() throws BadCommandArgumentsException, UsageException {
try {
coreAction.validate();
} catch (BadCommandArgumentsException e) {
StringBuilder badArgMsgBuilder = new StringBuilder();
badArgMsgBuilder.append(e.toString()).append("\n");
badArgMsgBuilder.append(usage(this, coreAction.getActionName()));
throw new BadCommandArgumentsException(badArgMsgBuilder.toString());
} catch (UsageException e) {
StringBuilder badArgMsgBuilder = new StringBuilder();
badArgMsgBuilder.append(e.toString()).append("\n");
badArgMsgBuilder.append(usage(this, coreAction.getActionName()));
throw new UsageException(badArgMsgBuilder.toString());
String badArgMsgBuilder =
e.toString() + "\n" + usage(this, coreAction.getActionName());
throw new BadCommandArgumentsException(badArgMsgBuilder);
}
}

View File

@ -16,9 +16,11 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.Parameter;
import org.apache.slider.common.params.AbstractArgsDelegate;
import org.apache.slider.common.params.DontSplitArguments;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import java.util.ArrayList;

View File

@ -17,7 +17,7 @@
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
/**
* Parameters sent by the Client to the AM
@ -43,7 +43,7 @@ public String getImage() {
* This is the URI in the FS to the Slider cluster; the conf file (and any
* other cluster-specifics) can be picked up here
*/
public String getAppDefDir() {
public String getAppDefPath() {
return createAction.sliderClusterURI;
}

View File

@ -16,16 +16,20 @@
* limitations under the License.
*/
package org.apache.slider.common.params;
package org.apache.hadoop.yarn.service.client.params;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.beust.jcommander.ParametersDelegate;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.slider.common.params.LaunchArgsAccessor;
import org.apache.slider.common.params.LaunchArgsDelegate;
import java.io.File;
@Parameters(commandNames = {SliderActions.ACTION_CREATE},
@Parameters(commandNames = { SliderActions.ACTION_CREATE},
commandDescription = SliderActions.DESCRIBE_ACTION_CREATE)
public class SliderAMCreateAction extends AbstractActionArgs implements
@ -48,8 +52,7 @@ public String getActionName() {
description = "URI to the Slider cluster", required = true)
public String sliderClusterURI;
@ParametersDelegate
LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
@ParametersDelegate LaunchArgsDelegate launchArgs = new LaunchArgsDelegate();
@Override
public String getRmAddress() {

View File

@ -0,0 +1,493 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.compinstance;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
import org.apache.hadoop.registry.client.types.ServiceRecord;
import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.client.api.NMClient;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.service.ServiceScheduler;
import org.apache.hadoop.yarn.service.component.Component;
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
import org.apache.hadoop.yarn.state.SingleArcTransition;
import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.BoundedAppender;
import org.apache.slider.api.resource.ContainerState;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher;
import org.apache.slider.server.servicemonitor.ProbeStatus;
import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.text.MessageFormat;
import java.util.Date;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import static org.apache.hadoop.yarn.api.records.ContainerExitStatus.KILLED_BY_APPMASTER;
import static org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE;
import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.*;
import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceState.*;
public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
Comparable<ComponentInstance> {
private static final Logger LOG =
LoggerFactory.getLogger(ComponentInstance.class);
private StateMachine<ComponentInstanceState, ComponentInstanceEventType,
ComponentInstanceEvent> stateMachine;
private Component component;
private final ReadLock readLock;
private final WriteLock writeLock;
private ComponentInstanceId compInstanceId = null;
private Path compInstanceDir;
private Container container;
private YarnRegistryViewForProviders yarnRegistryOperations;
private FileSystem fs;
private boolean timelineServiceEnabled = false;
private ServiceTimelinePublisher serviceTimelinePublisher;
private ServiceScheduler scheduler;
private BoundedAppender diagnostics = new BoundedAppender(64 * 1024);
private volatile ScheduledFuture containerStatusFuture;
private volatile ContainerStatus status;
private long containerStartedTime = 0;
// This container object is used for rest API query
private org.apache.slider.api.resource.Container containerSpec;
private static final StateMachineFactory<ComponentInstance,
ComponentInstanceState, ComponentInstanceEventType, ComponentInstanceEvent>
stateMachineFactory =
new StateMachineFactory<ComponentInstance, ComponentInstanceState,
ComponentInstanceEventType, ComponentInstanceEvent>(INIT)
.addTransition(INIT, RUNNING_BUT_UNREADY, STARTED,
new ContainerStartedTransition())
//From Running
.addTransition(RUNNING_BUT_UNREADY, INIT, STOP,
new ContainerStoppedTransition())
.addTransition(RUNNING_BUT_UNREADY, READY, BECOME_READY,
new ContainerBecomeReadyTransition())
// FROM READY
.addTransition(READY, RUNNING_BUT_UNREADY, BECOME_NOT_READY,
new ContainerBecomeNotReadyTransition())
.addTransition(READY, INIT, STOP, new ContainerStoppedTransition())
.installTopology();
public ComponentInstance(Component component,
ComponentInstanceId compInstanceId) {
this.stateMachine = stateMachineFactory.make(this);
this.component = component;
this.compInstanceId = compInstanceId;
this.scheduler = component.getScheduler();
this.yarnRegistryOperations =
component.getScheduler().getYarnRegistryOperations();
this.serviceTimelinePublisher =
component.getScheduler().getServiceTimelinePublisher();
if (YarnConfiguration
.timelineServiceV2Enabled(component.getScheduler().getConfig())) {
this.timelineServiceEnabled = true;
}
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
this.readLock = lock.readLock();
this.writeLock = lock.writeLock();
this.fs = scheduler.getContext().fs.getFileSystem();
}
private static class ContainerStartedTransition extends BaseTransition {
@Override public void transition(ComponentInstance compInstance,
ComponentInstanceEvent event) {
// Query container status for ip and host
compInstance.containerStatusFuture =
compInstance.scheduler.executorService.scheduleAtFixedRate(
new ContainerStatusRetriever(compInstance.scheduler,
compInstance.getContainerId(), compInstance), 0, 1,
TimeUnit.SECONDS);
org.apache.slider.api.resource.Container container =
new org.apache.slider.api.resource.Container();
container.setId(compInstance.getContainerId().toString());
container.setLaunchTime(new Date());
container.setState(org.apache.slider.api.resource.ContainerState.RUNNING_BUT_UNREADY);
container.setBareHost(compInstance.container.getNodeId().getHost());
container.setComponentName(compInstance.getCompInstanceName());
if (compInstance.containerSpec != null) {
// remove the previous container.
compInstance.getCompSpec().removeContainer(compInstance.containerSpec);
}
compInstance.containerSpec = container;
compInstance.getCompSpec().addContainer(container);
compInstance.containerStartedTime = System.currentTimeMillis();
if (compInstance.timelineServiceEnabled) {
compInstance.serviceTimelinePublisher
.componentInstanceStarted(container, compInstance);
}
}
}
private static class ContainerBecomeReadyTransition extends BaseTransition {
@Override
public void transition(ComponentInstance compInstance,
ComponentInstanceEvent event) {
compInstance.component.incContainersReady();
compInstance.containerSpec.setState(ContainerState.READY);
}
}
private static class ContainerBecomeNotReadyTransition extends BaseTransition {
@Override
public void transition(ComponentInstance compInstance,
ComponentInstanceEvent event) {
compInstance.component.decContainersReady();
compInstance.containerSpec.setState(ContainerState.RUNNING_BUT_UNREADY);
}
}
private static class ContainerStoppedTransition extends BaseTransition {
@Override
public void transition(ComponentInstance compInstance,
ComponentInstanceEvent event) {
// re-ask the failed container.
Component comp = compInstance.component;
comp.requestContainers(1);
LOG.info(compInstance.getCompInstanceId()
+ ": Container completed. Requested a new container." + System
.lineSeparator() + " exitStatus={}, diagnostics={}.",
event.getStatus().getExitStatus(),
event.getStatus().getDiagnostics());
String containerDiag =
compInstance.getCompInstanceId() + ": " + event.getStatus()
.getDiagnostics();
compInstance.diagnostics.append(containerDiag + System.lineSeparator());
boolean shouldExit = false;
// check if it exceeds the failure threshold
if (comp.currentContainerFailure > comp.maxContainerFailurePerComp) {
String exitDiag = MessageFormat.format(
"[COMPONENT {0}]: Failed {1} times, exceeded the limit - {2}. Shutting down now... "
+ System.lineSeparator(),
comp.getName(), comp.currentContainerFailure, comp.maxContainerFailurePerComp);
compInstance.diagnostics.append(exitDiag);
// append to global diagnostics that will be reported to RM.
comp.getScheduler().getDiagnostics().append(containerDiag);
comp.getScheduler().getDiagnostics().append(exitDiag);
LOG.warn(exitDiag);
shouldExit = true;
}
// clean up registry
// hdfs dir content will be overwritten when a new container gets started,
// so no need remove.
compInstance.scheduler.executorService
.submit(compInstance::cleanupRegistry);
// remove the failed ContainerId -> CompInstance mapping
comp.getScheduler().removeLiveCompInstance(event.getContainerId());
if (compInstance.timelineServiceEnabled) {
// record in ATS
compInstance.serviceTimelinePublisher
.componentInstanceFinished(compInstance,
event.getStatus().getExitStatus(), event.getStatus().getState(),
containerDiag);
}
compInstance.containerSpec.setState(ContainerState.STOPPED);
if (shouldExit) {
// Sleep for 5 seconds in hope that the state can be recorded in ATS.
// in case there's a client polling the comp state, it can be notified.
try {
Thread.sleep(5000);
} catch (InterruptedException e) {
LOG.error("Interrupted on sleep while exiting.", e);
}
ExitUtil.terminate(-1);
}
}
}
public ComponentInstanceState getState() {
this.readLock.lock();
try {
return this.stateMachine.getCurrentState();
} finally {
this.readLock.unlock();
}
}
@Override
public void handle(ComponentInstanceEvent event) {
try {
writeLock.lock();
ComponentInstanceState oldState = getState();
try {
stateMachine.doTransition(event.getType(), event);
} catch (InvalidStateTransitionException e) {
LOG.error("Invalid event " + event.getType() +
" at " + oldState + " for component instance " + compInstanceId, e);
}
if (oldState != getState()) {
LOG.info(getCompInstanceId() + " Transitioned from " + oldState + " to "
+ getState() + " on " + event.getType() + " event");
}
} finally {
writeLock.unlock();
}
}
public void setContainer(Container container) {
this.container = container;
this.compInstanceId.setContainerId(container.getId());
}
public String getCompInstanceName() {
return compInstanceId.getCompInstanceName();
}
public ContainerStatus getContainerStatus() {
return status;
}
public void updateContainerStatus(ContainerStatus status) {
this.status = status;
org.apache.slider.api.resource.Container container =
getCompSpec().getContainer(getContainerId().toString());
if (container != null) {
container.setIp(StringUtils.join(",", status.getIPs()));
container.setHostname(status.getHost());
if (timelineServiceEnabled) {
serviceTimelinePublisher.componentInstanceUpdated(container);
}
}
updateServiceRecord(yarnRegistryOperations, status);
}
public ContainerId getContainerId() {
return container.getId();
}
public String getCompName() {
return compInstanceId.getCompName();
}
public void setCompInstanceDir(Path dir) {
this.compInstanceDir = dir;
}
public Component getComponent() {
return component;
}
public Container getContainer() {
return container;
}
public ComponentInstanceId getCompInstanceId() {
return compInstanceId;
}
public NodeId getNodeId() {
return this.container.getNodeId();
}
public org.apache.slider.api.resource.Component getCompSpec() {
return component.getComponentSpec();
}
private static class BaseTransition implements
SingleArcTransition<ComponentInstance, ComponentInstanceEvent> {
@Override public void transition(ComponentInstance compInstance,
ComponentInstanceEvent event) {
}
}
public ProbeStatus ping() {
if (component.getProbe() == null) {
ProbeStatus status = new ProbeStatus();
status.setSuccess(true);
return status;
}
return component.getProbe().ping(this);
}
// Write service record into registry
private void updateServiceRecord(
YarnRegistryViewForProviders yarnRegistry, ContainerStatus status) {
ServiceRecord record = new ServiceRecord();
String containerId = status.getContainerId().toString();
record.set(YarnRegistryAttributes.YARN_ID, containerId);
record.description = getCompInstanceName();
record.set(YarnRegistryAttributes.YARN_PERSISTENCE,
PersistencePolicies.CONTAINER);
record.set("yarn:ip", status.getIPs());
record.set("yarn:hostname", status.getHost());
try {
yarnRegistry
.putComponent(RegistryPathUtils.encodeYarnID(containerId), record);
} catch (IOException e) {
LOG.error(
"Failed to update service record in registry: " + containerId + "");
}
}
// Release the container , cleanup registry, hdfs dir, and record in ATS
public void destroy() {
LOG.info(getCompInstanceId() + ": Flexed down by user, destroying.");
diagnostics.append(getCompInstanceId() + ": Flexed down by user");
if (container != null) {
scheduler.removeLiveCompInstance(container.getId());
component.getScheduler().getAmRMClient()
.releaseAssignedContainer(container.getId());
getCompSpec().removeContainer(containerSpec);
}
if (timelineServiceEnabled) {
serviceTimelinePublisher
.componentInstanceFinished(this, KILLED_BY_APPMASTER, COMPLETE,
diagnostics.toString());
}
scheduler.executorService.submit(this::cleanupRegistryAndCompHdfsDir);
}
private void cleanupRegistry() {
ContainerId containerId = getContainerId();
String cid = RegistryPathUtils.encodeYarnID(containerId.toString());
try {
yarnRegistryOperations.deleteComponent(getCompInstanceId(), cid);
} catch (IOException e) {
LOG.error(getCompInstanceId() + ": Failed to delete registry", e);
}
}
//TODO Maybe have a dedicated cleanup service.
public void cleanupRegistryAndCompHdfsDir() {
cleanupRegistry();
try {
if (compInstanceDir != null && fs.exists(compInstanceDir)) {
boolean deleted = fs.delete(compInstanceDir, true);
if (!deleted) {
LOG.error(getCompInstanceId()
+ ": Failed to delete component instance dir: "
+ compInstanceDir);
} else {
LOG.info(getCompInstanceId() + ": Deleted component instance dir: "
+ compInstanceDir);
}
}
} catch (IOException e) {
LOG.warn(getCompInstanceId() + ": Failed to delete directory", e);
}
}
// Query container status until ip and hostname are available and update
// the service record into registry service
private static class ContainerStatusRetriever implements Runnable {
private ContainerId containerId;
private NodeId nodeId;
private NMClient nmClient;
private ComponentInstance instance;
ContainerStatusRetriever(ServiceScheduler scheduler,
ContainerId containerId, ComponentInstance instance) {
this.containerId = containerId;
this.nodeId = instance.getNodeId();
this.nmClient = scheduler.getNmClient().getClient();
this.instance = instance;
}
@Override public void run() {
ContainerStatus status = null;
try {
status = nmClient.getContainerStatus(containerId, nodeId);
} catch (Exception e) {
if (e instanceof YarnException) {
throw new YarnRuntimeException(
instance.compInstanceId + " Failed to get container status on "
+ nodeId + " , cancelling.", e);
}
LOG.error(instance.compInstanceId + " Failed to get container status on "
+ nodeId + ", will try again", e);
return;
}
if (SliderUtils.isEmpty(status.getIPs()) || SliderUtils
.isUnset(status.getHost())) {
return;
}
instance.updateContainerStatus(status);
LOG.info(
instance.compInstanceId + " IP = " + status.getIPs() + ", host = "
+ status.getHost() + ", cancel container status retriever");
instance.containerStatusFuture.cancel(false);
}
}
@Override
public int compareTo(ComponentInstance to) {
long delta = containerStartedTime - to.containerStartedTime;
if (delta == 0) {
return getCompInstanceId().compareTo(to.getCompInstanceId());
} else if (delta < 0) {
return -1;
} else {
return 1;
}
}
@Override public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
ComponentInstance instance = (ComponentInstance) o;
if (containerStartedTime != instance.containerStartedTime)
return false;
return compInstanceId.equals(instance.compInstanceId);
}
@Override public int hashCode() {
int result = compInstanceId.hashCode();
result = 31 * result + (int) (containerStartedTime ^ (containerStartedTime
>>> 32));
return result;
}
}

View File

@ -0,0 +1,58 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.compinstance;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.event.AbstractEvent;
public class ComponentInstanceEvent
extends AbstractEvent<ComponentInstanceEventType> {
private ContainerId id;
private ContainerStatus status;
private boolean shouldDestroy = false;
public ComponentInstanceEvent(ContainerId containerId,
ComponentInstanceEventType componentInstanceEventType) {
super(componentInstanceEventType);
this.id = containerId;
}
public ContainerId getContainerId() {
return id;
}
public ContainerStatus getStatus() {
return this.status;
}
public ComponentInstanceEvent setStatus(ContainerStatus status) {
this.status = status;
return this;
}
public void setShouldDestroy() {
shouldDestroy = true;
}
public boolean shouldDestroy() {
return shouldDestroy;
}
}

View File

@ -1,4 +1,4 @@
/*
/**
* 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
@ -15,12 +15,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* ATS tests
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.slider.server.appmaster.timelineservice;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
package org.apache.hadoop.yarn.service.compinstance;
public enum ComponentInstanceEventType {
STARTED,
STOP,
BECOME_READY,
BECOME_NOT_READY
}

View File

@ -0,0 +1,91 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.compinstance;
import org.apache.hadoop.yarn.api.records.ContainerId;
public class ComponentInstanceId implements Comparable<ComponentInstanceId> {
private long Id;
private String name;
private ContainerId containerId;
public ComponentInstanceId(long id, String name) {
Id = id;
this.name = name;
}
public long getId() {
return Id;
}
public String getCompName() {
return name;
}
public String getCompInstanceName() {
return getCompName() + "-" + getId();
}
public void setContainerId(ContainerId containerId) {
this.containerId = containerId;
}
@Override
public String toString() {
if (containerId == null) {
return "[COMPINSTANCE " + getCompInstanceName() + "]";
} else {
return "[COMPINSTANCE " + getCompInstanceName() + " : " + containerId + "]";
}
}
@Override public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
ComponentInstanceId that = (ComponentInstanceId) o;
if (getId() != that.getId())
return false;
return getCompName() != null ? getCompName().equals(that.getCompName()) :
that.getCompName() == null;
}
@Override public int hashCode() {
int result = (int) (getId() ^ (getId() >>> 32));
result = 31 * result + (getCompName() != null ? getCompName().hashCode() : 0);
return result;
}
@Override
public int compareTo(ComponentInstanceId to) {
int delta = this.getCompName().compareTo(to.getCompName());
if (delta == 0) {
return Long.compare(this.getId(), to.getId());
} else if (delta < 0) {
return -1;
} else {
return 1;
}
}
}

View File

@ -0,0 +1,26 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.compinstance;
public enum ComponentInstanceState {
INIT,
RUNNING_BUT_UNREADY,
READY,
UPGRADING
}

View File

@ -0,0 +1,487 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.component;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceId;
import org.apache.hadoop.yarn.service.ContainerFailureTracker;
import org.apache.hadoop.yarn.service.ServiceContext;
import org.apache.hadoop.yarn.service.ServiceScheduler;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEvent;
import org.apache.hadoop.yarn.service.metrics.ServiceMetrics;
import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
import org.apache.hadoop.yarn.state.MultipleArcTransition;
import org.apache.hadoop.yarn.state.SingleArcTransition;
import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Apps;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.server.servicemonitor.MonitorUtils;
import org.apache.slider.server.servicemonitor.Probe;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import static org.apache.hadoop.yarn.api.records.ContainerExitStatus.*;
import static org.apache.hadoop.yarn.service.component.ComponentEventType.*;
import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.STARTED;
import static org.apache.hadoop.yarn.service.compinstance.ComponentInstanceEventType.STOP;
import static org.apache.hadoop.yarn.service.component.ComponentState.*;
import static org.apache.slider.api.ResourceKeys.CONTAINER_FAILURE_THRESHOLD;
public class Component implements EventHandler<ComponentEvent> {
private static final Logger LOG = LoggerFactory.getLogger(Component.class);
private org.apache.slider.api.resource.Component componentSpec;
private long allocateId;
private Priority priority;
private ServiceMetrics componentMetrics;
private ServiceScheduler scheduler;
private ServiceContext context;
private AMRMClientAsync<ContainerRequest> amrmClient;
private AtomicLong instanceIdCounter = new AtomicLong();
private Map<ComponentInstanceId, ComponentInstance> compInstances =
new ConcurrentHashMap<>();
// component instances to be assigned with a container
private List<ComponentInstance> pendingInstances = new LinkedList<>();
private ContainerFailureTracker failureTracker;
private Probe probe;
private final ReentrantReadWriteLock.ReadLock readLock;
private final ReentrantReadWriteLock.WriteLock writeLock;
public int maxContainerFailurePerComp;
// The number of containers failed since last reset. This excludes preempted,
// disk_failed containers etc. This will be reset to 0 periodically.
public volatile int currentContainerFailure;
private StateMachine<ComponentState, ComponentEventType, ComponentEvent>
stateMachine;
private AsyncDispatcher compInstanceDispatcher;
private static final StateMachineFactory<Component, ComponentState, ComponentEventType, ComponentEvent>
stateMachineFactory =
new StateMachineFactory<Component, ComponentState, ComponentEventType, ComponentEvent>(
INIT)
// INIT will only got to FLEXING
.addTransition(INIT, EnumSet.of(STABLE, FLEXING),
FLEX, new FlexComponentTransition())
// container allocated by RM
.addTransition(FLEXING, FLEXING, CONTAINER_ALLOCATED,
new ContainerAllocatedTransition())
// container launched on NM
.addTransition(FLEXING, EnumSet.of(STABLE, FLEXING),
CONTAINER_STARTED, new ContainerStartedTransition())
// container failed while flexing
.addTransition(FLEXING, FLEXING, CONTAINER_COMPLETED,
new ContainerCompletedTransition())
// Flex while previous flex is still in progress
.addTransition(FLEXING, EnumSet.of(FLEXING), FLEX,
new FlexComponentTransition())
// container failed while stable
.addTransition(STABLE, FLEXING, CONTAINER_COMPLETED,
new ContainerCompletedTransition())
// Ignore surplus container
.addTransition(STABLE, STABLE, CONTAINER_ALLOCATED,
new ContainerAllocatedTransition())
// Flex by user
// For flex up, go to FLEXING state
// For flex down, go to STABLE state
.addTransition(STABLE, EnumSet.of(STABLE, FLEXING),
FLEX, new FlexComponentTransition())
.installTopology();
public Component(org.apache.slider.api.resource.Component component,
long allocateId, ServiceContext context) {
this.allocateId = allocateId;
this.priority = Priority.newInstance((int) allocateId);
this.componentSpec = component;
componentMetrics = ServiceMetrics.register(component.getName(),
"Metrics for component " + component.getName());
componentMetrics
.tag("type", "Metrics type [component or service]", "component");
this.scheduler = context.scheduler;
this.context = context;
amrmClient = scheduler.getAmRMClient();
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
this.readLock = lock.readLock();
this.writeLock = lock.writeLock();
this.stateMachine = stateMachineFactory.make(this);
compInstanceDispatcher = scheduler.getCompInstanceDispatcher();
failureTracker =
new ContainerFailureTracker(context, this);
probe = MonitorUtils.getProbe(componentSpec.getReadinessCheck());
maxContainerFailurePerComp = componentSpec.getConfiguration()
.getPropertyInt(CONTAINER_FAILURE_THRESHOLD, 10);
createNumCompInstances(component.getNumberOfContainers());
}
private void createNumCompInstances(long count) {
for (int i = 0; i < count; i++) {
createOneCompInstance();
}
}
private void createOneCompInstance() {
ComponentInstanceId id =
new ComponentInstanceId(instanceIdCounter.getAndIncrement(),
componentSpec.getName());
ComponentInstance instance = new ComponentInstance(this, id);
compInstances.put(id, instance);
pendingInstances.add(instance);
}
private static class FlexComponentTransition implements
MultipleArcTransition<Component, ComponentEvent, ComponentState> {
// For flex up, go to FLEXING state
// For flex down, go to STABLE state
@Override
public ComponentState transition(Component component,
ComponentEvent event) {
component.setDesiredContainers((int)event.getDesired());
if (!component.areDependenciesReady()) {
LOG.info("[FLEX COMPONENT {}]: Flex deferred because dependencies not"
+ " satisfied.", component.getName());
return component.getState();
}
if (component.getState() == INIT) {
// This happens on init
LOG.info("[INIT COMPONENT " + component.getName() + "]: " + event
.getDesired() + " instances.");
component.requestContainers(event.getDesired());
return FLEXING;
}
long before = component.getComponentSpec().getNumberOfContainers();
long delta = event.getDesired() - before;
component.getComponentSpec().setNumberOfContainers(event.getDesired());
if (delta > 0) {
// Scale up
LOG.info("[FLEX UP COMPONENT " + component.getName() + "]: scaling up from "
+ before + " to " + event.getDesired());
component.requestContainers(delta);
component.createNumCompInstances(delta);
return FLEXING;
} else if (delta < 0){
delta = 0 - delta;
// scale down
LOG.info("[FLEX DOWN COMPONENT " + component.getName()
+ "]: scaling down from " + before + " to " + event.getDesired());
List<ComponentInstance> list =
new ArrayList<>(component.compInstances.values());
// sort in Most recent -> oldest order, destroy most recent ones.
Collections.sort(list, Collections.reverseOrder());
for (int i = 0; i < delta; i++) {
ComponentInstance instance = list.get(i);
// remove the instance
component.compInstances.remove(instance.getCompInstanceId());
component.pendingInstances.remove(instance);
component.componentMetrics.containersFailed.incr();
component.componentMetrics.containersRunning.decr();
// decrement id counter
component.instanceIdCounter.decrementAndGet();
instance.destroy();
}
return STABLE;
} else {
LOG.info("[FLEX COMPONENT " + component.getName() + "]: already has " +
event.getDesired() + " instances, ignoring");
return STABLE;
}
}
}
private static class ContainerAllocatedTransition extends BaseTransition {
@Override
public void transition(Component component, ComponentEvent event) {
component.assignContainerToCompInstance(event.getContainer());
}
}
private static class ContainerStartedTransition implements
MultipleArcTransition<Component,ComponentEvent,ComponentState> {
@Override public ComponentState transition(Component component,
ComponentEvent event) {
component.compInstanceDispatcher.getEventHandler().handle(
new ComponentInstanceEvent(event.getInstance().getContainerId(),
STARTED));
component.incRunningContainers();
return checkIfStable(component);
}
}
private static ComponentState checkIfStable(Component component) {
// if desired == running
if (component.componentMetrics.containersRunning.value() == component
.getComponentSpec().getNumberOfContainers()) {
return STABLE;
} else {
return FLEXING;
}
}
private static class ContainerCompletedTransition extends BaseTransition {
@Override
public void transition(Component component, ComponentEvent event) {
component.updateMetrics(event.getStatus());
// add back to pending list
component.pendingInstances.add(event.getInstance());
LOG.info(
"[COMPONENT {}]: {} completed, num pending comp instances increased to {}.",
component.getName(), event.getStatus().getContainerId(),
component.pendingInstances.size());
component.compInstanceDispatcher.getEventHandler().handle(
new ComponentInstanceEvent(event.getStatus().getContainerId(),
STOP).setStatus(event.getStatus()));
}
}
public ServiceMetrics getCompMetrics () {
return componentMetrics;
}
private void assignContainerToCompInstance(Container container) {
if (pendingInstances.size() == 0) {
LOG.info(
"[COMPONENT {}]: No pending component instance left, release surplus container {}",
getName(), container.getId());
scheduler.getAmRMClient().releaseAssignedContainer(container.getId());
componentMetrics.surplusContainers.incr();
scheduler.getServiceMetrics().surplusContainers.incr();
return;
}
ComponentInstance instance = pendingInstances.remove(0);
LOG.info(
"[COMPONENT {}]: {} allocated, num pending component instances reduced to {}",
getName(), container.getId(), pendingInstances.size());
instance.setContainer(container);
scheduler.addLiveCompInstance(container.getId(), instance);
LOG.info(
"[COMPONENT {}]: Assigned {} to component instance {} and launch on host {} ",
getName(), container.getId(), instance.getCompInstanceName(),
container.getNodeId());
scheduler.getContainerLaunchService()
.launchCompInstance(scheduler.getApp(), instance, container);
}
@SuppressWarnings({ "unchecked" })
public void requestContainers(long count) {
Resource resource = Resource
.newInstance(componentSpec.getResource().getMemoryMB(),
componentSpec.getResource().getCpus());
for (int i = 0; i < count; i++) {
//TODO Once YARN-5468 is done, use that for anti-affinity
ContainerRequest request =
ContainerRequest.newBuilder().capability(resource).priority(priority)
.allocationRequestId(allocateId).relaxLocality(true).build();
amrmClient.addContainerRequest(request);
}
}
private void setDesiredContainers(int n) {
int delta = n - scheduler.getServiceMetrics().containersDesired.value();
if (delta > 0) {
scheduler.getServiceMetrics().containersDesired.incr(delta);
} else {
scheduler.getServiceMetrics().containersDesired.decr(delta);
}
componentMetrics.containersDesired.set(n);
}
private void updateMetrics(ContainerStatus status) {
switch (status.getExitStatus()) {
case SUCCESS:
componentMetrics.containersSucceeded.incr();
scheduler.getServiceMetrics().containersSucceeded.incr();
return;
case PREEMPTED:
componentMetrics.containersPreempted.incr();
scheduler.getServiceMetrics().containersPreempted.incr();
break;
case DISKS_FAILED:
componentMetrics.containersDiskFailure.incr();
scheduler.getServiceMetrics().containersDiskFailure.incr();
break;
default:
break;
}
// containersFailed include preempted, disks_failed etc.
componentMetrics.containersFailed.incr();
scheduler.getServiceMetrics().containersFailed.incr();
// dec running container
decRunningContainers();
if (Apps.shouldCountTowardsNodeBlacklisting(status.getExitStatus())) {
String host = scheduler.getLiveInstances().get(status.getContainerId())
.getNodeId().getHost();
failureTracker.incNodeFailure(host);
currentContainerFailure++ ;
}
}
public boolean areDependenciesReady() {
List<String> dependencies = componentSpec.getDependencies();
if (SliderUtils.isEmpty(dependencies)) {
return true;
}
for (String dependency : dependencies) {
Component dependentComponent =
scheduler.getAllComponents().get(dependency);
if (dependentComponent == null) {
LOG.error("Couldn't find dependency {} for {} (should never happen)",
dependency, getName());
continue;
}
if (dependentComponent.getNumReadyInstances() < dependentComponent
.getNumDesiredInstances()) {
LOG.info("[COMPONENT {}]: Dependency {} not satisfied, only {} of {}"
+ " instances are ready.", getName(), dependency,
dependentComponent.getNumReadyInstances(),
dependentComponent.getNumDesiredInstances());
return false;
}
}
return true;
}
private void incRunningContainers() {
componentMetrics.containersRunning.incr();
scheduler.getServiceMetrics().containersRunning.incr();
}
public void incContainersReady() {
componentMetrics.containersReady.incr();
}
public void decContainersReady() {
componentMetrics.containersReady.decr();
}
private void decRunningContainers() {
componentMetrics.containersRunning.decr();
scheduler.getServiceMetrics().containersRunning.decr();
}
public int getNumReadyInstances() {
return componentMetrics.containersReady.value();
}
public int getNumRunningInstances() {
return componentMetrics.containersRunning.value();
}
public int getNumDesiredInstances() {
return componentMetrics.containersDesired.value();
}
public Map<ComponentInstanceId, ComponentInstance> getAllComponentInstances() {
return compInstances;
}
public org.apache.slider.api.resource.Component getComponentSpec() {
return this.componentSpec;
}
public void resetCompFailureCount() {
LOG.info("[COMPONENT {}]: Reset container failure count from {} to 0.",
getName(), currentContainerFailure);
currentContainerFailure = 0;
failureTracker.resetContainerFailures();
}
public Probe getProbe() {
return probe;
}
public Priority getPriority() {
return priority;
}
public String getName () {
return componentSpec.getName();
}
public ComponentState getState() {
this.readLock.lock();
try {
return this.stateMachine.getCurrentState();
} finally {
this.readLock.unlock();
}
}
public ServiceScheduler getScheduler() {
return scheduler;
}
@Override
public void handle(ComponentEvent event) {
try {
writeLock.lock();
ComponentState oldState = getState();
try {
stateMachine.doTransition(event.getType(), event);
} catch (InvalidStateTransitionException e) {
LOG.error("Invalid event " + event.getType() +
" at " + oldState + " for component " + componentSpec.getName(), e);
}
if (oldState != getState()) {
LOG.info("[COMPONENT {}] Transitioned from {} to {} on {} event.",
componentSpec.getName(), oldState, getState(), event.getType());
}
} finally {
writeLock.unlock();
}
}
private static class BaseTransition implements
SingleArcTransition<Component, ComponentEvent> {
@Override public void transition(Component component,
ComponentEvent event) {
}
}
public ServiceContext getContext() {
return context;
}
}

View File

@ -0,0 +1,83 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.component;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
public class ComponentEvent extends AbstractEvent<ComponentEventType> {
private long desired;
private final String name;
private final ComponentEventType type;
private Container container;
private ComponentInstance instance;
private ContainerStatus status;
public ComponentEvent(String name, ComponentEventType type) {
super(type);
this.name = name;
this.type = type;
}
public String getName() {
return name;
}
public ComponentEventType getType() {
return type;
}
public long getDesired() {
return desired;
}
public ComponentEvent setDesired(long desired) {
this.desired = desired;
return this;
}
public Container getContainer() {
return container;
}
public ComponentEvent setContainer(Container container) {
this.container = container;
return this;
}
public ComponentInstance getInstance() {
return instance;
}
public ComponentEvent setInstance(ComponentInstance instance) {
this.instance = instance;
return this;
}
public ContainerStatus getStatus() {
return status;
}
public ComponentEvent setStatus(ContainerStatus status) {
this.status = status;
return this;
}
}

View File

@ -0,0 +1,26 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.component;
public enum ComponentEventType {
FLEX,
CONTAINER_ALLOCATED,
CONTAINER_STARTED,
CONTAINER_COMPLETED
}

View File

@ -0,0 +1,25 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.component;
public enum ComponentState {
INIT,
FLEXING,
STABLE
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.common;
package org.apache.hadoop.yarn.service.conf;
/**
* These are the keys that can be added to <code>conf/slider-client.xml</code>.
@ -187,4 +187,5 @@ public interface SliderXmlConfKeys {
String DFS_NAMENODE_DU_RESERVED_KEY = "dfs.namenode.resource.du.reserved";
String MAPREDUCE_JOB_CREDENTIALS_BINARY = "mapreduce.job.credentials.binary";
}

View File

@ -0,0 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.conf;
public interface YarnServiceConfKeys {
// Retry settings for the ServiceClient to talk to Service AppMaster
String CLIENT_AM_RETRY_MAX_WAIT_MS = "yarn.service.client-am.retry.max-wait-ms";
String CLIENT_AM_RETRY_MAX_INTERVAL_MS = "yarn.service.client-am.retry-interval-ms";
}

View File

@ -0,0 +1,91 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.impl.pb.client;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.service.ClientAMProtocol;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
import org.apache.hadoop.yarn.service.impl.pb.service.ClientAMProtocolPB;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto;
public class ClientAMProtocolPBClientImpl
implements ClientAMProtocol, Closeable {
private ClientAMProtocolPB proxy;
public ClientAMProtocolPBClientImpl(long clientVersion,
InetSocketAddress addr, Configuration conf) throws IOException {
RPC.setProtocolEngine(conf, ClientAMProtocolPB.class,
ProtobufRpcEngine.class);
proxy = RPC.getProxy(ClientAMProtocolPB.class, clientVersion, addr, conf);
}
@Override public FlexComponentsResponseProto flexComponents(
FlexComponentsRequestProto request) throws IOException, YarnException {
try {
return proxy.flexComponents(null, request);
} catch (ServiceException e) {
RPCUtil.unwrapAndThrowException(e);
}
return null;
}
@Override
public GetStatusResponseProto getStatus(GetStatusRequestProto request)
throws IOException, YarnException {
try {
return proxy.getStatus(null, request);
} catch (ServiceException e) {
RPCUtil.unwrapAndThrowException(e);
}
return null;
}
@Override
public StopResponseProto stop(StopRequestProto requestProto)
throws IOException, YarnException {
try {
return proxy.stop(null, requestProto);
} catch (ServiceException e) {
RPCUtil.unwrapAndThrowException(e);
}
return null;
}
@Override public void close() {
if (this.proxy != null) {
RPC.stopProxy(this.proxy);
}
}
}

View File

@ -0,0 +1,29 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.impl.pb.service;
import org.apache.hadoop.ipc.ProtocolInfo;
import org.apache.hadoop.yarn.proto.ClientAMProtocol;
@ProtocolInfo(
protocolName = "org.apache.hadoop.yarn.service.ClientAMProtocol",
protocolVersion = 1)
public interface ClientAMProtocolPB extends
ClientAMProtocol.ClientAMProtocolService.BlockingInterface {
}

View File

@ -0,0 +1,70 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.impl.pb.service;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
import org.apache.hadoop.yarn.service.ClientAMProtocol;
import java.io.IOException;
public class ClientAMProtocolPBServiceImpl implements ClientAMProtocolPB {
private ClientAMProtocol real;
public ClientAMProtocolPBServiceImpl(ClientAMProtocol impl) {
this.real = impl;
}
@Override
public FlexComponentsResponseProto flexComponents(RpcController controller,
FlexComponentsRequestProto request) throws ServiceException {
try {
return real.flexComponents(request);
} catch (IOException | YarnException e) {
throw new ServiceException(e);
}
}
@Override public GetStatusResponseProto getStatus(RpcController controller,
GetStatusRequestProto request) throws ServiceException {
try {
return real.getStatus(request);
} catch (IOException | YarnException e) {
throw new ServiceException(e);
}
}
@Override
public org.apache.hadoop.yarn.proto.ClientAMProtocol.StopResponseProto stop(
RpcController controller,
org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto request)
throws ServiceException {
try {
return real.stop(request);
} catch (IOException | YarnException e) {
throw new ServiceException(e);
}
}
}

View File

@ -16,26 +16,21 @@
* limitations under the License.
*/
package org.apache.slider.server.appmaster.metrics;
package org.apache.hadoop.yarn.service.metrics;
import com.codahale.metrics.Counter;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsSource;
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
import java.util.HashMap;
import java.util.Map;
import static org.apache.hadoop.metrics2.lib.Interns.info;
@Metrics(context = "yarn-native-service")
public class SliderMetrics implements MetricsSource {
public class ServiceMetrics implements MetricsSource {
@Metric("containers requested")
public MutableGaugeInt containersRequested;
@ -46,18 +41,18 @@ public class SliderMetrics implements MetricsSource {
@Metric("containers running")
public MutableGaugeInt containersRunning;
@Metric("containers ready")
public MutableGaugeInt containersReady;
@Metric("containers desired")
public MutableGaugeInt containersDesired;
@Metric("containers completed")
public MutableGaugeInt containersCompleted;
@Metric("containers succeeded")
public MutableGaugeInt containersSucceeded;
@Metric("containers failed")
public MutableGaugeInt containersFailed;
@Metric("containers failed since last threshold")
public MutableGaugeInt failedSinceLastThreshold;
@Metric("containers preempted")
public MutableGaugeInt containersPreempted;
@ -72,7 +67,7 @@ public class SliderMetrics implements MetricsSource {
protected final MetricsRegistry registry;
public SliderMetrics(MetricsInfo metricsInfo) {
public ServiceMetrics(MetricsInfo metricsInfo) {
registry = new MetricsRegistry(metricsInfo);
}
@ -81,8 +76,8 @@ public void getMetrics(MetricsCollector collector, boolean all) {
registry.snapshot(collector.addRecord(registry.info()), all);
}
public static SliderMetrics register(String name, String description) {
SliderMetrics metrics = new SliderMetrics(info(name, description));
public static ServiceMetrics register(String name, String description) {
ServiceMetrics metrics = new ServiceMetrics(info(name, description));
DefaultMetricsSystem.instance().register(name, description, metrics);
return metrics;
}
@ -92,14 +87,13 @@ public void tag(String name, String description, String value) {
}
@Override public String toString() {
return "SliderMetrics{"
return "ServiceMetrics{"
+ "containersRequested=" + containersRequested.value()
+ ", pendingAAContainers=" + pendingAAContainers.value()
+ ", containersRunning=" + containersRunning.value()
+ ", containersDesired=" + containersDesired.value()
+ ", containersCompleted=" + containersCompleted.value()
+ ", containersSucceeded=" + containersSucceeded.value()
+ ", containersFailed=" + containersFailed.value()
+ ", failedSinceLastThreshold=" + failedSinceLastThreshold.value()
+ ", containersPreempted=" + containersPreempted.value()
+ ", surplusContainers=" + surplusContainers.value() + '}';
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.providers;
package org.apache.hadoop.yarn.service.provider;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
@ -125,29 +125,4 @@ public void validateConfigFiles(List<ConfigFile> configFiles,
validateConfigFile(file, fs);
}
}
/**
* Process client operations for applications such as install, configure.
* @param fileSystem
* @param registryOperations
* @param configuration
* @param operation
* @param clientInstallPath
* @param clientPackage
* @param clientConfig
* @param name
* @throws SliderException
*/
public void processClientOperation(SliderFileSystem fileSystem,
RegistryOperations registryOperations,
Configuration configuration,
String operation,
File clientInstallPath,
File clientPackage,
JSONObject clientConfig,
String name)
throws SliderException {
throw new SliderException("Provider does not support client operations.");
}
}

View File

@ -0,0 +1,97 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.provider;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.Component;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.exceptions.SliderException;
import org.apache.slider.core.launch.AbstractLauncher;
import org.apache.slider.core.launch.CommandLineBuilder;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.ServiceContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Map;
import java.util.Map.Entry;
import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$;
public abstract class AbstractProviderService implements ProviderService,
SliderKeys {
protected static final Logger log =
LoggerFactory.getLogger(AbstractProviderService.class);
public abstract void processArtifact(AbstractLauncher launcher,
ComponentInstance compInstance, SliderFileSystem fileSystem,
Application application)
throws IOException;
public void buildContainerLaunchContext(AbstractLauncher launcher,
Application application, ComponentInstance instance,
SliderFileSystem fileSystem) throws IOException, SliderException {
Component component = instance.getComponent().getComponentSpec();;
processArtifact(launcher, instance, fileSystem, application);
ServiceContext context =
instance.getComponent().getScheduler().getContext();
// Generate tokens (key-value pair) for config substitution.
// Get pre-defined tokens
Map<String, String> globalTokens =
instance.getComponent().getScheduler().globalTokens;
Map<String, String> tokensForSubstitution = ProviderUtils
.initCompTokensForSubstitute(instance);
tokensForSubstitution.putAll(globalTokens);
// Set the environment variables in launcher
launcher.putEnv(SliderUtils
.buildEnvMap(component.getConfiguration(), tokensForSubstitution));
launcher.setEnv("WORK_DIR", ApplicationConstants.Environment.PWD.$());
launcher.setEnv("LOG_DIR", ApplicationConstants.LOG_DIR_EXPANSION_VAR);
if (System.getenv(HADOOP_USER_NAME) != null) {
launcher.setEnv(HADOOP_USER_NAME, System.getenv(HADOOP_USER_NAME));
}
launcher.setEnv("LANG", "en_US.UTF-8");
launcher.setEnv("LC_ALL", "en_US.UTF-8");
launcher.setEnv("LANGUAGE", "en_US.UTF-8");
for (Entry<String, String> entry : launcher.getEnv().entrySet()) {
tokensForSubstitution.put($(entry.getKey()), entry.getValue());
}
//TODO add component host tokens?
// ProviderUtils.addComponentHostTokens(tokensForSubstitution, amState);
// create config file on hdfs and add local resource
ProviderUtils.createConfigFileAndAddLocalResource(launcher, fileSystem,
component, tokensForSubstitution, instance, context);
// substitute launch command
String launchCommand = ProviderUtils
.substituteStrWithTokens(component.getLaunchCommand(),
tokensForSubstitution);
CommandLineBuilder operation = new CommandLineBuilder();
operation.add(launchCommand);
operation.addOutAndErrFiles(OUT_FILE, ERR_FILE);
launcher.addCommand(operation.build());
}
}

View File

@ -16,23 +16,24 @@
* limitations under the License.
*/
package org.apache.slider.providers;
package org.apache.hadoop.yarn.service.provider;
import org.apache.hadoop.yarn.service.provider.defaultImpl.DefaultProviderFactory;
import org.apache.slider.api.resource.Artifact;
import org.apache.slider.core.exceptions.SliderException;
import org.apache.slider.providers.docker.DockerProviderFactory;
import org.apache.slider.providers.tarball.TarballProviderFactory;
import org.apache.hadoop.yarn.service.provider.docker.DockerProviderFactory;
import org.apache.hadoop.yarn.service.provider.tarball.TarballProviderFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Base class for factories.
*/
public abstract class SliderProviderFactory {
public abstract class ProviderFactory {
protected static final Logger LOG =
LoggerFactory.getLogger(SliderProviderFactory.class);
LoggerFactory.getLogger(ProviderFactory.class);
protected SliderProviderFactory() {}
protected ProviderFactory() {}
public abstract AbstractClientProvider createClientProvider();
@ -52,9 +53,8 @@ public static synchronized AbstractClientProvider getClientProvider(Artifact
* Create a provider for a specific application
* @param artifact artifact
* @return provider factory
* @throws SliderException on any instantiation problem
*/
public static synchronized SliderProviderFactory createSliderProviderFactory(
public static synchronized ProviderFactory createSliderProviderFactory(
Artifact artifact) {
if (artifact == null || artifact.getType() == null) {
LOG.debug("Loading service provider type default");

View File

@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.service.provider;
import org.apache.slider.api.resource.Application;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.core.exceptions.SliderException;
import org.apache.slider.core.launch.AbstractLauncher;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import java.io.IOException;
public interface ProviderService {
/**
* Set up the entire container launch context
*/
void buildContainerLaunchContext(AbstractLauncher containerLauncher,
Application application, ComponentInstance instance,
SliderFileSystem sliderFileSystem) throws IOException, SliderException;
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.providers;
package org.apache.hadoop.yarn.service.provider;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
@ -24,10 +24,6 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
import org.apache.hadoop.registry.client.types.ServiceRecord;
import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
@ -38,21 +34,22 @@
import org.apache.slider.api.resource.Component;
import org.apache.slider.api.resource.ConfigFile;
import org.apache.slider.api.resource.Configuration;
import org.apache.slider.common.SliderKeys;
import org.apache.slider.common.SliderXmlConfKeys;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.NoSuchNodeException;
import org.apache.slider.core.exceptions.SliderException;
import org.apache.slider.core.launch.AbstractLauncher;
import org.apache.slider.core.launch.ContainerLauncher;
import org.apache.slider.core.registry.docstore.ConfigFormat;
import org.apache.slider.core.registry.docstore.PublishedConfiguration;
import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
import org.apache.slider.server.appmaster.state.RoleInstance;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.ServiceContext;
import org.apache.slider.server.appmaster.state.StateAccessForProviders;
import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.FileNotFoundException;
@ -68,7 +65,7 @@
import java.util.regex.Pattern;
import static org.apache.slider.api.ServiceApiConstants.*;
import static org.apache.slider.util.ServiceApiUtil.$;
import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$;
/**
* This is a factoring out of methods handy for providers. It's bonded to a log
@ -76,16 +73,9 @@
*/
public class ProviderUtils implements RoleKeys, SliderKeys {
protected final Logger log;
protected static final Logger log =
LoggerFactory.getLogger(ProviderUtils.class);
/**
* Create an instance
* @param log log directory to use -usually the provider
*/
public ProviderUtils(Logger log) {
this.log = log;
}
/**
* Add oneself to the classpath. This does not work
@ -169,7 +159,7 @@ public static String substituteStrWithTokens(String content,
}
// configs will be substituted by corresponding env in tokenMap
public void substituteMapWithTokens(Map<String, String> configs,
public static void substituteMapWithTokens(Map<String, String> configs,
Map<String, String> tokenMap) {
for (Map.Entry<String, String> entry : configs.entrySet()) {
String value = entry.getValue();
@ -256,26 +246,28 @@ public void localizeServiceKeytabs(ContainerLauncher launcher,
// 1. Create all config files for a component on hdfs for localization
// 2. Add the config file to localResource
public synchronized void createConfigFileAndAddLocalResource(
ContainerLauncher launcher, SliderFileSystem fs, Component component,
Map<String, String> tokensForSubstitution, RoleInstance roleInstance,
StateAccessForProviders appState) throws IOException {
public static synchronized void createConfigFileAndAddLocalResource(
AbstractLauncher launcher, SliderFileSystem fs, Component component,
Map<String, String> tokensForSubstitution, ComponentInstance instance,
ServiceContext context) throws IOException {
Path compDir =
new Path(new Path(fs.getAppDir(), "components"), component.getName());
Path compInstanceDir =
new Path(compDir, roleInstance.getCompInstanceName());
new Path(compDir, instance.getCompInstanceName());
if (!fs.getFileSystem().exists(compInstanceDir)) {
log.info(instance.getCompInstanceId() + ": Creating dir on hdfs: " + compInstanceDir);
fs.getFileSystem().mkdirs(compInstanceDir,
new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
roleInstance.compInstanceDir = compInstanceDir;
log.info("Creating component instance dir: " + compInstanceDir);
instance.setCompInstanceDir(compInstanceDir);
} else {
log.info("Component instance conf dir already exists: " + compInstanceDir);
}
log.info("Tokens substitution for component: " + roleInstance
if (log.isDebugEnabled()) {
log.debug("Tokens substitution for component instance: " + instance
.getCompInstanceName() + System.lineSeparator()
+ tokensForSubstitution);
}
for (ConfigFile originalFile : component.getConfiguration().getFiles()) {
ConfigFile configFile = originalFile.copy();
@ -289,7 +281,7 @@ public synchronized void createConfigFileAndAddLocalResource(
Path remoteFile = new Path(compInstanceDir, fileName);
if (!fs.getFileSystem().exists(remoteFile)) {
log.info("Saving config file on hdfs for component " + roleInstance
log.info("Saving config file on hdfs for component " + instance
.getCompInstanceName() + ": " + configFile);
if (configFile.getSrcFile() != null) {
@ -298,12 +290,12 @@ public synchronized void createConfigFileAndAddLocalResource(
case HADOOP_XML:
// Hadoop_xml_template
resolveHadoopXmlTemplateAndSaveOnHdfs(fs.getFileSystem(),
tokensForSubstitution, configFile, remoteFile, appState);
tokensForSubstitution, configFile, remoteFile, context);
break;
case TEMPLATE:
// plain-template
resolvePlainTemplateAndSaveOnHdfs(fs.getFileSystem(),
tokensForSubstitution, configFile, remoteFile, appState);
tokensForSubstitution, configFile, remoteFile, context);
break;
default:
log.info("Not supporting loading src_file for " + configFile);
@ -312,7 +304,7 @@ public synchronized void createConfigFileAndAddLocalResource(
} else {
// non-template
resolveNonTemplateConfigsAndSaveOnHdfs(fs, tokensForSubstitution,
roleInstance, configFile, fileName, remoteFile);
instance, configFile, fileName, remoteFile);
}
}
@ -335,8 +327,8 @@ public synchronized void createConfigFileAndAddLocalResource(
}
}
private void resolveNonTemplateConfigsAndSaveOnHdfs(SliderFileSystem fs,
Map<String, String> tokensForSubstitution, RoleInstance roleInstance,
private static void resolveNonTemplateConfigsAndSaveOnHdfs(SliderFileSystem fs,
Map<String, String> tokensForSubstitution, ComponentInstance instance,
ConfigFile configFile, String fileName, Path remoteFile)
throws IOException {
// substitute non-template configs
@ -356,7 +348,7 @@ private void resolveNonTemplateConfigsAndSaveOnHdfs(SliderFileSystem fs,
os.flush();
}
} else {
log.info("Component instance = " + roleInstance.getCompInstanceName()
log.info("Component instance = " + instance.getCompInstanceName()
+ ", config file already exists: " + remoteFile);
}
}
@ -364,13 +356,12 @@ private void resolveNonTemplateConfigsAndSaveOnHdfs(SliderFileSystem fs,
// 1. substitute config template - only handle hadoop_xml format
// 2. save on hdfs
@SuppressWarnings("unchecked")
private void resolveHadoopXmlTemplateAndSaveOnHdfs(FileSystem fs,
private static void resolveHadoopXmlTemplateAndSaveOnHdfs(FileSystem fs,
Map<String, String> tokensForSubstitution, ConfigFile configFile,
Path remoteFile, StateAccessForProviders appState) throws IOException {
Path remoteFile, ServiceContext context) throws IOException {
Map<String, String> conf;
try {
conf = (Map<String, String>) appState.getConfigFileCache()
.get(configFile);
conf = (Map<String, String>) context.configCache.get(configFile);
} catch (ExecutionException e) {
log.info("Failed to load config file: " + configFile, e);
return;
@ -407,12 +398,12 @@ private void resolveHadoopXmlTemplateAndSaveOnHdfs(FileSystem fs,
// 1) read the template as a string
// 2) do token substitution
// 3) save on hdfs
private void resolvePlainTemplateAndSaveOnHdfs(FileSystem fs,
private static void resolvePlainTemplateAndSaveOnHdfs(FileSystem fs,
Map<String, String> tokensForSubstitution, ConfigFile configFile,
Path remoteFile, StateAccessForProviders appState) {
Path remoteFile, ServiceContext context) {
String content;
try {
content = (String) appState.getConfigFileCache().get(configFile);
content = (String) context.configCache.get(configFile);
} catch (ExecutionException e) {
log.info("Failed to load config file: " + configFile, e);
return;
@ -429,17 +420,18 @@ private void resolvePlainTemplateAndSaveOnHdfs(FileSystem fs,
/**
* Get initial component token map to be substituted into config values.
* @param roleInstance role instance
* @return tokens to replace
*/
public Map<String, String> initCompTokensForSubstitute(
RoleInstance roleInstance) {
public static Map<String, String> initCompTokensForSubstitute(
ComponentInstance instance) {
Map<String, String> tokens = new HashMap<>();
tokens.put(COMPONENT_NAME, roleInstance.role);
tokens.put(COMPONENT_NAME_LC, roleInstance.role.toLowerCase());
tokens.put(COMPONENT_INSTANCE_NAME, roleInstance.getCompInstanceName());
tokens.put(CONTAINER_ID, roleInstance.getContainerId().toString());
tokens.put(COMPONENT_ID, String.valueOf(roleInstance.componentId));
tokens.put(COMPONENT_NAME, instance.getCompSpec().getName());
tokens
.put(COMPONENT_NAME_LC, instance.getCompSpec().getName().toLowerCase());
tokens.put(COMPONENT_INSTANCE_NAME, instance.getCompInstanceName());
tokens.put(CONTAINER_ID, instance.getContainer().getId().toString());
tokens.put(COMPONENT_ID,
String.valueOf(instance.getCompInstanceId().getId()));
return tokens;
}
@ -448,7 +440,7 @@ public Map<String, String> initCompTokensForSubstitute(
* @param tokens existing tokens
* @param amState access to AM state
*/
public void addComponentHostTokens(Map<String, String> tokens,
public static void addComponentHostTokens(Map<String, String> tokens,
StateAccessForProviders amState) {
if (amState == null) {
return;
@ -468,7 +460,7 @@ public void addComponentHostTokens(Map<String, String> tokens,
* @param hostOnly whether host or host/server name will be added to list
* @return list of hosts
*/
public Iterable<String> getHostsList(Collection<ClusterNode> values,
public static Iterable<String> getHostsList(Collection<ClusterNode> values,
boolean hostOnly) {
List<String> hosts = new ArrayList<>();
for (ClusterNode cn : values) {
@ -476,50 +468,4 @@ public Iterable<String> getHostsList(Collection<ClusterNode> values,
}
return hosts;
}
/**
* Update ServiceRecord in Registry with IP and hostname.
* @param amState access to AM state
* @param yarnRegistry acces to YARN registry
* @param containerId container ID
* @param roleName component name
* @param ip list of IPs
* @param hostname hostname
*/
public void updateServiceRecord(StateAccessForProviders amState,
YarnRegistryViewForProviders yarnRegistry,
String containerId, String roleName, List<String> ip, String hostname)
throws IOException {
try {
RoleInstance role = null;
if(ip != null && !ip.isEmpty()){
role = amState.getOwnedContainer(containerId);
role.ip = ip.get(0);
}
if(hostname != null && !hostname.isEmpty()){
role = amState.getOwnedContainer(containerId);
role.hostname = hostname;
}
if (role != null) {
// create and publish updated service record (including hostname & ip)
ServiceRecord record = new ServiceRecord();
record.set(YarnRegistryAttributes.YARN_ID, containerId);
record.description = role.getCompInstanceName();
record.set(YarnRegistryAttributes.YARN_PERSISTENCE,
PersistencePolicies.CONTAINER);
// TODO: use constants from YarnRegistryAttributes
if (role.ip != null) {
record.set("yarn:ip", role.ip);
}
if (role.hostname != null) {
record.set("yarn:hostname", role.hostname);
}
yarnRegistry.putComponent(
RegistryPathUtils.encodeYarnID(containerId), record);
}
} catch (NoSuchNodeException e) {
// ignore - there is nothing to do if we don't find a container
log.warn("Owned container {} not found - {}", containerId, e);
}
}
}

View File

@ -15,9 +15,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers;
package org.apache.hadoop.yarn.service.provider.defaultImpl;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.slider.api.resource.Artifact;
import org.apache.slider.api.resource.ConfigFile;

View File

@ -15,10 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers;
package org.apache.hadoop.yarn.service.provider.defaultImpl;
public final class DefaultProviderFactory extends SliderProviderFactory {
private static final SliderProviderFactory FACTORY = new
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.hadoop.yarn.service.provider.ProviderService;
import org.apache.hadoop.yarn.service.provider.ProviderFactory;
public final class DefaultProviderFactory extends ProviderFactory {
private static final ProviderFactory FACTORY = new
DefaultProviderFactory();
private DefaultProviderFactory() {}
@ -41,7 +45,7 @@ public ProviderService createServerProvider() {
return Server.PROVIDER;
}
public static SliderProviderFactory getInstance() {
public static ProviderFactory getInstance() {
return FACTORY;
}
}

View File

@ -15,24 +15,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers;
package org.apache.hadoop.yarn.service.provider.defaultImpl;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.provider.AbstractProviderService;
import org.apache.slider.api.resource.Application;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.core.launch.ContainerLauncher;
import org.apache.slider.server.appmaster.state.RoleInstance;
import org.apache.slider.core.launch.AbstractLauncher;
import java.io.IOException;
public class DefaultProviderService extends AbstractProviderService {
protected DefaultProviderService() {
super(DefaultProviderService.class.getSimpleName());
}
@Override
public void processArtifact(ContainerLauncher launcher, Application
application, RoleInstance roleInstance, SliderFileSystem fileSystem)
public void processArtifact(AbstractLauncher launcher,
ComponentInstance compInstance, SliderFileSystem fileSystem,
Application application)
throws IOException {
}
}

View File

@ -15,14 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers.docker;
package org.apache.hadoop.yarn.service.provider.docker;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.slider.api.resource.Artifact;
import org.apache.slider.api.resource.ConfigFile;
import org.apache.slider.common.SliderKeys;
import org.apache.slider.providers.AbstractClientProvider;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.slider.util.RestApiErrorMessages;
import java.io.IOException;

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers.docker;
package org.apache.hadoop.yarn.service.provider.docker;
public interface DockerKeys {
String PROVIDER_DOCKER = "docker";

View File

@ -15,14 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers.docker;
package org.apache.hadoop.yarn.service.provider.docker;
import org.apache.slider.providers.AbstractClientProvider;
import org.apache.slider.providers.ProviderService;
import org.apache.slider.providers.SliderProviderFactory;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.hadoop.yarn.service.provider.ProviderService;
import org.apache.hadoop.yarn.service.provider.ProviderFactory;
public class DockerProviderFactory extends SliderProviderFactory {
private static final SliderProviderFactory FACTORY = new
public class DockerProviderFactory extends ProviderFactory {
private static final ProviderFactory FACTORY = new
DockerProviderFactory();
private DockerProviderFactory() {
@ -46,7 +46,7 @@ public ProviderService createServerProvider() {
return Server.PROVIDER;
}
public static SliderProviderFactory getInstance() {
public static ProviderFactory getInstance() {
return FACTORY;
}
}

View File

@ -15,18 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers.docker;
package org.apache.hadoop.yarn.service.provider.docker;
import org.apache.hadoop.registry.client.api.RegistryConstants;
import org.apache.hadoop.registry.client.binding.RegistryUtils;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.provider.AbstractProviderService;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.Component;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.core.launch.ContainerLauncher;
import org.apache.slider.providers.AbstractProviderService;
import org.apache.slider.server.appmaster.state.RoleInstance;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.slider.core.launch.AbstractLauncher;
import java.io.IOException;
import java.text.MessageFormat;
@ -34,33 +31,27 @@
public class DockerProviderService extends AbstractProviderService
implements DockerKeys {
protected static final Logger log =
LoggerFactory.getLogger(DockerProviderService.class);
protected DockerProviderService() {
super(DockerProviderService.class.getSimpleName());
}
public void processArtifact(ContainerLauncher launcher, Application
application, RoleInstance roleInstance, SliderFileSystem fileSystem)
throws IOException {
Component component = roleInstance.providerRole.component;
public void processArtifact(AbstractLauncher launcher,
ComponentInstance compInstance, SliderFileSystem fileSystem,
Application application) throws IOException{
launcher.setYarnDockerMode(true);
launcher.setDockerImage(component.getArtifact().getId());
launcher.setDockerNetwork(component.getConfiguration()
launcher.setDockerImage(compInstance.getCompSpec().getArtifact().getId());
launcher.setDockerNetwork(compInstance.getCompSpec().getConfiguration()
.getProperty(DOCKER_NETWORK, DEFAULT_DOCKER_NETWORK));
String domain = getConfig().get(RegistryConstants.KEY_DNS_DOMAIN);
String domain = compInstance.getComponent().getScheduler().getConfig()
.get(RegistryConstants.KEY_DNS_DOMAIN);
String hostname;
if (domain == null || domain.isEmpty()) {
hostname = MessageFormat.format("{0}.{1}.{2}", roleInstance
.getCompInstanceName(), application.getName(), RegistryUtils
.currentUser());
hostname = MessageFormat
.format("{0}.{1}.{2}", compInstance.getCompInstanceName(),
application.getName(), RegistryUtils.currentUser());
} else {
hostname = MessageFormat.format("{0}.{1}.{2}.{3}", roleInstance
.getCompInstanceName(), application.getName(), RegistryUtils
.currentUser(), domain);
hostname = MessageFormat
.format("{0}.{1}.{2}.{3}", compInstance.getCompInstanceName(),
application.getName(), RegistryUtils.currentUser(), domain);
}
launcher.setDockerHostname(hostname);
launcher.setRunPrivilegedContainer(component.getRunPrivilegedContainer());
launcher.setRunPrivilegedContainer(
compInstance.getCompSpec().getRunPrivilegedContainer());
}
}

View File

@ -15,15 +15,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers.tarball;
package org.apache.hadoop.yarn.service.provider.tarball;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.slider.api.resource.Artifact;
import org.apache.slider.api.resource.ConfigFile;
import org.apache.slider.common.SliderKeys;
import org.apache.slider.providers.AbstractClientProvider;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.slider.util.RestApiErrorMessages;
import java.io.IOException;

View File

@ -15,14 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers.tarball;
package org.apache.hadoop.yarn.service.provider.tarball;
import org.apache.slider.providers.AbstractClientProvider;
import org.apache.slider.providers.ProviderService;
import org.apache.slider.providers.SliderProviderFactory;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.hadoop.yarn.service.provider.ProviderService;
import org.apache.hadoop.yarn.service.provider.ProviderFactory;
public class TarballProviderFactory extends SliderProviderFactory {
private static final SliderProviderFactory FACTORY = new
public class TarballProviderFactory extends ProviderFactory {
private static final ProviderFactory FACTORY = new
TarballProviderFactory();
private TarballProviderFactory() {
@ -46,7 +46,7 @@ public ProviderService createServerProvider() {
return Server.PROVIDER;
}
public static SliderProviderFactory getInstance() {
public static ProviderFactory getInstance() {
return FACTORY;
}
}

View File

@ -15,40 +15,34 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.slider.providers.tarball;
package org.apache.hadoop.yarn.service.provider.tarball;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.provider.AbstractProviderService;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.Component;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.core.launch.ContainerLauncher;
import org.apache.slider.providers.AbstractProviderService;
import org.apache.slider.server.appmaster.state.RoleInstance;
import org.apache.slider.core.launch.AbstractLauncher;
import java.io.IOException;
public class TarballProviderService extends AbstractProviderService {
protected TarballProviderService() {
super(TarballProviderService.class.getSimpleName());
}
@Override
public void processArtifact(ContainerLauncher launcher, Application
application, RoleInstance roleInstance, SliderFileSystem fileSystem)
public void processArtifact(AbstractLauncher launcher,
ComponentInstance instance, SliderFileSystem fileSystem,
Application application)
throws IOException {
Path artifact = new Path(roleInstance.providerRole.component
.getArtifact().getId());
Path artifact = new Path(instance.getCompSpec().getArtifact().getId());
if (!fileSystem.isFile(artifact)) {
throw new IOException("Package doesn't exist as a resource: " +
artifact.toString());
throw new IOException(
"Package doesn't exist as a resource: " + artifact.toString());
}
log.info("Adding resource {}", artifact.toString());
LocalResourceType type = LocalResourceType.ARCHIVE;
LocalResource packageResource = fileSystem.createAmResource(
artifact, type);
LocalResource packageResource = fileSystem.createAmResource(artifact, type);
launcher.addLocalResource(APP_LIB_DIR, packageResource);
}
}

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.server.appmaster.timelineservice;
package org.apache.hadoop.yarn.service.timelineservice;
import org.apache.commons.configuration2.SubsetConfiguration;
import org.apache.hadoop.metrics2.MetricsRecord;
@ -30,23 +30,23 @@
* hadoop-metrics2 property files. Specifically, you would create this class by
* adding the following to by This would actually be set as: <code>
* [prefix].sink.[some instance name].class
* =org.apache.slider.server.appmaster.timelineservice.SliderMetricsSink
* =org.apache.hadoop.yarn.service.timelineservice.ServiceMetricsSink
* </code>, where <tt>prefix</tt> is "atsv2": and <tt>some instance name</tt> is
* just any unique name, so properties can be differentiated if there are
* multiple sinks of the same type created
*/
public class SliderMetricsSink implements MetricsSink {
public class ServiceMetricsSink implements MetricsSink {
private static final Logger log =
LoggerFactory.getLogger(SliderMetricsSink.class);
LoggerFactory.getLogger(ServiceMetricsSink.class);
private ServiceTimelinePublisher serviceTimelinePublisher;
public SliderMetricsSink() {
public ServiceMetricsSink() {
}
public SliderMetricsSink(ServiceTimelinePublisher publisher) {
public ServiceMetricsSink(ServiceTimelinePublisher publisher) {
serviceTimelinePublisher = publisher;
}
@ -81,14 +81,14 @@ public void putMetrics(MetricsRecord record) {
log.debug("Publishing service metrics. " + record);
}
serviceTimelinePublisher.publishMetrics(record.metrics(), appId,
SliderTimelineEntityType.SERVICE_ATTEMPT.toString(),
ServiceTimelineEntityType.SERVICE_ATTEMPT.toString(),
record.timestamp());
} else if (isComponentMetrics) {
if (log.isDebugEnabled()) {
log.debug("Publishing Component metrics. " + record);
}
serviceTimelinePublisher.publishMetrics(record.metrics(), record.name(),
SliderTimelineEntityType.COMPONENT.toString(), record.timestamp());
ServiceTimelineEntityType.COMPONENT.toString(), record.timestamp());
}
}

View File

@ -16,12 +16,12 @@
* limitations under the License.
*/
package org.apache.slider.server.appmaster.timelineservice;
package org.apache.hadoop.yarn.service.timelineservice;
/**
* Slider entities that are published to ATS.
*/
public enum SliderTimelineEntityType {
public enum ServiceTimelineEntityType {
/**
* Used for publishing service entity information.
*/

View File

@ -16,12 +16,12 @@
* limitations under the License.
*/
package org.apache.slider.server.appmaster.timelineservice;
package org.apache.hadoop.yarn.service.timelineservice;
/**
* Events that are used to store in ATS.
*/
public enum SliderTimelineEvent {
public enum ServiceTimelineEvent {
SERVICE_ATTEMPT_REGISTERED,
SERVICE_ATTEMPT_UNREGISTERED,

View File

@ -16,12 +16,12 @@
* limitations under the License.
*/
package org.apache.slider.server.appmaster.timelineservice;
package org.apache.hadoop.yarn.service.timelineservice;
/**
* Constants which are stored as key in ATS
*/
public final class SliderTimelineMetricsConstants {
public final class ServiceTimelineMetricsConstants {
public static final String URI = "URI";
@ -75,6 +75,8 @@ public final class SliderTimelineMetricsConstants {
public static final String COMPONENT_NAME = "COMPONENT_NAME";
public static final String COMPONENT_INSTANCE_NAME = "COMPONENT_INSTANCE_NAME";
/**
* component constants.
*/

View File

@ -16,18 +16,13 @@
* limitations under the License.
*/
package org.apache.slider.server.appmaster.timelineservice;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
package org.apache.hadoop.yarn.service.timelineservice;
import org.apache.hadoop.metrics2.AbstractMetric;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
@ -40,11 +35,21 @@
import org.apache.slider.api.resource.Container;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.server.appmaster.actions.ActionStopSlider;
import org.apache.hadoop.yarn.service.compinstance.ComponentInstance;
import org.apache.hadoop.yarn.service.ServiceContext;
import org.apache.slider.server.appmaster.state.AppState;
import org.apache.slider.server.appmaster.state.RoleInstance;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
/**
* A single service that publishes all the Timeline Entities.
*/
@ -60,9 +65,17 @@ public class ServiceTimelinePublisher extends CompositeService {
private static final Logger log =
LoggerFactory.getLogger(ServiceTimelinePublisher.class);
@Override
protected void serviceInit(org.apache.hadoop.conf.Configuration configuration)
throws Exception {
addService(timelineClient);
}
@Override
protected void serviceStop() throws Exception {
stopped = true;
super.serviceStop();
}
public boolean isStopped() {
@ -74,8 +87,7 @@ public ServiceTimelinePublisher(TimelineV2Client client) {
timelineClient = client;
}
public void serviceAttemptRegistered(AppState appState) {
Application application = appState.getClusterStatus();
public void serviceAttemptRegistered(Application application) {
long currentTimeMillis = application.getLaunchTime() == null
? System.currentTimeMillis() : application.getLaunchTime().getTime();
@ -84,16 +96,18 @@ public void serviceAttemptRegistered(AppState appState) {
// create info keys
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(SliderTimelineMetricsConstants.NAME, application.getName());
entityInfos.put(SliderTimelineMetricsConstants.STATE,
entityInfos.put(ServiceTimelineMetricsConstants.NAME, application.getName());
entityInfos.put(ServiceTimelineMetricsConstants.STATE,
application.getState().toString());
entityInfos.put(SliderTimelineMetricsConstants.LAUNCH_TIME,
entityInfos.put(ServiceTimelineMetricsConstants.LAUNCH_TIME,
currentTimeMillis);
entity.addInfo(ServiceTimelineMetricsConstants.QUICK_LINKS,
application.getQuicklinks());
entity.addInfo(entityInfos);
// add an event
TimelineEvent startEvent = new TimelineEvent();
startEvent.setId(SliderTimelineEvent.SERVICE_ATTEMPT_REGISTERED.toString());
startEvent.setId(ServiceTimelineEvent.SERVICE_ATTEMPT_REGISTERED.toString());
startEvent.setTimestamp(currentTimeMillis);
entity.addEvent(startEvent);
@ -102,7 +116,7 @@ public void serviceAttemptRegistered(AppState appState) {
// publish application specific configurations
publishConfigurations(application.getConfiguration(), application.getId(),
SliderTimelineEntityType.SERVICE_ATTEMPT.toString(), true);
ServiceTimelineEntityType.SERVICE_ATTEMPT.toString(), true);
// publish component as separate entity.
publishComponents(application.getComponents());
@ -110,11 +124,29 @@ public void serviceAttemptRegistered(AppState appState) {
public void serviceAttemptUpdated(Application application) {
TimelineEntity entity = createServiceAttemptEntity(application.getId());
entity.addInfo(SliderTimelineMetricsConstants.QUICK_LINKS,
entity.addInfo(ServiceTimelineMetricsConstants.QUICK_LINKS,
application.getQuicklinks());
putEntity(entity);
}
public void serviceAttemptUnregistered(ServiceContext context) {
TimelineEntity entity = createServiceAttemptEntity(
context.attemptId.getApplicationId().toString());
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(ServiceTimelineMetricsConstants.STATE,
FinalApplicationStatus.FAILED);
entity.addInfo(entityInfos);
// add an event
TimelineEvent finishEvent = new TimelineEvent();
finishEvent
.setId(ServiceTimelineEvent.SERVICE_ATTEMPT_UNREGISTERED.toString());
finishEvent.setTimestamp(System.currentTimeMillis());
entity.addEvent(finishEvent);
putEntity(entity);
}
public void serviceAttemptUnregistered(AppState appState,
ActionStopSlider stopAction) {
long currentTimeMillis = System.currentTimeMillis();
@ -124,16 +156,16 @@ public void serviceAttemptUnregistered(AppState appState,
// add info
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(SliderTimelineMetricsConstants.EXIT_STATUS_CODE,
entityInfos.put(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE,
stopAction.getExitCode());
entityInfos.put(SliderTimelineMetricsConstants.STATE,
entityInfos.put(ServiceTimelineMetricsConstants.STATE,
stopAction.getFinalApplicationStatus().toString());
if (stopAction.getMessage() != null) {
entityInfos.put(SliderTimelineMetricsConstants.EXIT_REASON,
entityInfos.put(ServiceTimelineMetricsConstants.EXIT_REASON,
stopAction.getMessage());
}
if (stopAction.getEx() != null) {
entityInfos.put(SliderTimelineMetricsConstants.DIAGNOSTICS_INFO,
entityInfos.put(ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO,
stopAction.getEx().toString());
}
entity.addInfo(entityInfos);
@ -141,7 +173,7 @@ public void serviceAttemptUnregistered(AppState appState,
// add an event
TimelineEvent startEvent = new TimelineEvent();
startEvent
.setId(SliderTimelineEvent.SERVICE_ATTEMPT_UNREGISTERED.toString());
.setId(ServiceTimelineEvent.SERVICE_ATTEMPT_UNREGISTERED.toString());
startEvent.setTimestamp(currentTimeMillis);
entity.addEvent(startEvent);
@ -149,27 +181,29 @@ public void serviceAttemptUnregistered(AppState appState,
}
public void componentInstanceStarted(Container container,
String componentName) {
ComponentInstance instance) {
TimelineEntity entity = createComponentInstanceEntity(container.getId());
entity.setCreatedTime(container.getLaunchTime().getTime());
// create info keys
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(SliderTimelineMetricsConstants.BARE_HOST,
entityInfos.put(ServiceTimelineMetricsConstants.BARE_HOST,
container.getBareHost());
entityInfos.put(SliderTimelineMetricsConstants.STATE,
entityInfos.put(ServiceTimelineMetricsConstants.STATE,
container.getState().toString());
entityInfos.put(SliderTimelineMetricsConstants.LAUNCH_TIME,
entityInfos.put(ServiceTimelineMetricsConstants.LAUNCH_TIME,
container.getLaunchTime().getTime());
entityInfos.put(SliderTimelineMetricsConstants.COMPONENT_NAME,
componentName);
entityInfos.put(ServiceTimelineMetricsConstants.COMPONENT_NAME,
instance.getCompName());
entityInfos.put(ServiceTimelineMetricsConstants.COMPONENT_INSTANCE_NAME,
instance.getCompInstanceName());
entity.addInfo(entityInfos);
// add an event
TimelineEvent startEvent = new TimelineEvent();
startEvent
.setId(SliderTimelineEvent.COMPONENT_INSTANCE_REGISTERED.toString());
.setId(ServiceTimelineEvent.COMPONENT_INSTANCE_REGISTERED.toString());
startEvent.setTimestamp(container.getLaunchTime().getTime());
entity.addEvent(startEvent);
@ -181,40 +215,62 @@ public void componentInstanceFinished(RoleInstance instance) {
// create info keys
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(SliderTimelineMetricsConstants.EXIT_STATUS_CODE,
entityInfos.put(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE,
instance.exitCode);
entityInfos.put(SliderTimelineMetricsConstants.DIAGNOSTICS_INFO,
entityInfos.put(ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO,
instance.diagnostics);
// TODO need to change the state based on enum value.
entityInfos.put(SliderTimelineMetricsConstants.STATE, "FINISHED");
entityInfos.put(ServiceTimelineMetricsConstants.STATE, "FINISHED");
entity.addInfo(entityInfos);
// add an event
TimelineEvent startEvent = new TimelineEvent();
startEvent
.setId(SliderTimelineEvent.COMPONENT_INSTANCE_UNREGISTERED.toString());
.setId(ServiceTimelineEvent.COMPONENT_INSTANCE_UNREGISTERED.toString());
startEvent.setTimestamp(System.currentTimeMillis());
entity.addEvent(startEvent);
putEntity(entity);
}
public void componentInstanceUpdated(Container container,
String componentName) {
public void componentInstanceFinished(ComponentInstance instance,
int exitCode, ContainerState state, String diagnostics) {
TimelineEntity entity = createComponentInstanceEntity(
instance.getContainer().getId().toString());
// create info keys
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE,
exitCode);
entityInfos.put(ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO, diagnostics);
entityInfos.put(ServiceTimelineMetricsConstants.STATE, state);
entity.addInfo(entityInfos);
// add an event
TimelineEvent startEvent = new TimelineEvent();
startEvent
.setId(ServiceTimelineEvent.COMPONENT_INSTANCE_UNREGISTERED.toString());
startEvent.setTimestamp(System.currentTimeMillis());
entity.addEvent(startEvent);
putEntity(entity);
}
public void componentInstanceUpdated(Container container) {
TimelineEntity entity = createComponentInstanceEntity(container.getId());
// create info keys
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(SliderTimelineMetricsConstants.IP, container.getIp());
entityInfos.put(SliderTimelineMetricsConstants.HOSTNAME,
entityInfos.put(ServiceTimelineMetricsConstants.IP, container.getIp());
entityInfos.put(ServiceTimelineMetricsConstants.HOSTNAME,
container.getHostname());
entityInfos.put(SliderTimelineMetricsConstants.STATE,
entityInfos.put(ServiceTimelineMetricsConstants.STATE,
container.getState().toString());
entity.addInfo(entityInfos);
TimelineEvent updateEvent = new TimelineEvent();
updateEvent
.setId(SliderTimelineEvent.COMPONENT_INSTANCE_UPDATED.toString());
.setId(ServiceTimelineEvent.COMPONENT_INSTANCE_UPDATED.toString());
updateEvent.setTimestamp(System.currentTimeMillis());
entity.addEvent(updateEvent);
@ -229,29 +285,29 @@ private void publishComponents(List<Component> components) {
// create info keys
Map<String, Object> entityInfos = new HashMap<String, Object>();
entityInfos.put(SliderTimelineMetricsConstants.ARTIFACT_ID,
entityInfos.put(ServiceTimelineMetricsConstants.ARTIFACT_ID,
component.getArtifact().getId());
entityInfos.put(SliderTimelineMetricsConstants.ARTIFACT_TYPE,
entityInfos.put(ServiceTimelineMetricsConstants.ARTIFACT_TYPE,
component.getArtifact().getType().toString());
if (component.getResource().getProfile() != null) {
entityInfos.put(SliderTimelineMetricsConstants.RESOURCE_PROFILE,
entityInfos.put(ServiceTimelineMetricsConstants.RESOURCE_PROFILE,
component.getResource().getProfile());
}
entityInfos.put(SliderTimelineMetricsConstants.RESOURCE_CPU,
entityInfos.put(ServiceTimelineMetricsConstants.RESOURCE_CPU,
component.getResource().getCpus());
entityInfos.put(SliderTimelineMetricsConstants.RESOURCE_MEMORY,
entityInfos.put(ServiceTimelineMetricsConstants.RESOURCE_MEMORY,
component.getResource().getMemory());
if (component.getLaunchCommand() != null) {
entityInfos.put(SliderTimelineMetricsConstants.LAUNCH_COMMAND,
entityInfos.put(ServiceTimelineMetricsConstants.LAUNCH_COMMAND,
component.getLaunchCommand());
}
entityInfos.put(SliderTimelineMetricsConstants.UNIQUE_COMPONENT_SUPPORT,
entityInfos.put(ServiceTimelineMetricsConstants.UNIQUE_COMPONENT_SUPPORT,
component.getUniqueComponentSupport().toString());
entityInfos.put(SliderTimelineMetricsConstants.RUN_PRIVILEGED_CONTAINER,
entityInfos.put(ServiceTimelineMetricsConstants.RUN_PRIVILEGED_CONTAINER,
component.getRunPrivilegedContainer().toString());
if (component.getPlacementPolicy() != null) {
entityInfos.put(SliderTimelineMetricsConstants.PLACEMENT_POLICY,
entityInfos.put(ServiceTimelineMetricsConstants.PLACEMENT_POLICY,
component.getPlacementPolicy().getLabel());
}
entity.addInfo(entityInfos);
@ -260,7 +316,7 @@ private void publishComponents(List<Component> components) {
// publish component specific configurations
publishConfigurations(component.getConfiguration(), component.getName(),
SliderTimelineEntityType.COMPONENT.toString(), false);
ServiceTimelineEntityType.COMPONENT.toString(), false);
}
}
@ -307,7 +363,7 @@ private void publishConfigurations(Iterator<Entry<String, String>> iterator,
}
/**
* Called from SliderMetricsSink at regular interval of time.
* Called from ServiceMetricsSink at regular interval of time.
* @param metrics of service or components
* @param entityId Id of entity
* @param entityType Type of entity
@ -329,19 +385,19 @@ public void publishMetrics(Iterable<AbstractMetric> metrics, String entityId,
private TimelineEntity createServiceAttemptEntity(String serviceId) {
TimelineEntity entity = createTimelineEntity(serviceId,
SliderTimelineEntityType.SERVICE_ATTEMPT.toString());
ServiceTimelineEntityType.SERVICE_ATTEMPT.toString());
return entity;
}
private TimelineEntity createComponentInstanceEntity(String instanceId) {
TimelineEntity entity = createTimelineEntity(instanceId,
SliderTimelineEntityType.COMPONENT_INSTANCE.toString());
ServiceTimelineEntityType.COMPONENT_INSTANCE.toString());
return entity;
}
private TimelineEntity createComponentEntity(String componentId) {
TimelineEntity entity = createTimelineEntity(componentId,
SliderTimelineEntityType.COMPONENT.toString());
ServiceTimelineEntityType.COMPONENT.toString());
return entity;
}

View File

@ -21,7 +21,7 @@
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.slider.server.appmaster.timelineservice;
package org.apache.hadoop.yarn.service.timelineservice;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.slider.util;
package org.apache.hadoop.yarn.service.utils;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang.StringUtils;
@ -24,6 +24,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.registry.client.api.RegistryConstants;
import org.apache.hadoop.registry.client.binding.RegistryUtils;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.slider.api.resource.Application;
import org.apache.slider.api.resource.Artifact;
import org.apache.slider.api.resource.Component;
@ -32,9 +33,12 @@
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.persist.JsonSerDeser;
import org.apache.slider.providers.AbstractClientProvider;
import org.apache.slider.providers.SliderProviderFactory;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.hadoop.yarn.service.provider.ProviderFactory;
import org.apache.slider.server.servicemonitor.MonitorUtils;
import org.apache.slider.server.services.utility.PatternValidator;
import org.apache.slider.util.RestApiConstants;
import org.apache.slider.util.RestApiErrorMessages;
import org.codehaus.jackson.map.PropertyNamingStrategy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -51,9 +55,11 @@
public class ServiceApiUtil {
private static final Logger LOG =
LoggerFactory.getLogger(ServiceApiUtil.class);
private static JsonSerDeser<Application> jsonSerDeser =
public static JsonSerDeser<Application> jsonSerDeser =
new JsonSerDeser<>(Application.class,
PropertyNamingStrategy.CAMEL_CASE_TO_LOWER_CASE_WITH_UNDERSCORES);
private static final PatternValidator compNamePattern
= new PatternValidator("[a-z][a-z0-9-]*");
@VisibleForTesting
public static void setJsonSerDeser(JsonSerDeser jsd) {
@ -224,7 +230,9 @@ public static void validateAndResolveApplication(Application application,
public static void validateComponent(Component comp, FileSystem fs)
throws IOException {
AbstractClientProvider compClientProvider = SliderProviderFactory
validateCompName(comp.getName());
AbstractClientProvider compClientProvider = ProviderFactory
.getClientProvider(comp.getArtifact());
compClientProvider.validateArtifact(comp.getArtifact(), fs);
@ -248,6 +256,20 @@ public static void validateComponent(Component comp, FileSystem fs)
MonitorUtils.getProbe(comp.getReadinessCheck());
}
// Check component name format and transform to lower case.
public static void validateCompName(String compName) {
if (StringUtils.isEmpty(compName)) {
throw new IllegalArgumentException("Component name can not be empty");
}
// validate component name
if (compName.contains("_")) {
throw new IllegalArgumentException(
"Invalid format for component name: " + compName
+ ", can not use '_' as DNS hostname does not allow underscore. Use '-' Instead. ");
}
compNamePattern.validate(compName);
}
@VisibleForTesting
public static List<Component> getApplicationComponents(SliderFileSystem
fs, String appName) throws IOException {
@ -258,9 +280,13 @@ public static Application loadApplication(SliderFileSystem fs, String
appName) throws IOException {
Path appJson = getAppJsonPath(fs, appName);
LOG.info("Loading application definition from " + appJson);
Application externalApplication = jsonSerDeser.load(fs.getFileSystem(),
appJson);
return externalApplication;
return jsonSerDeser.load(fs.getFileSystem(), appJson);
}
public static Application loadApplicationFrom(SliderFileSystem fs,
Path appDefPath) throws IOException {
LOG.info("Loading application definition from " + appDefPath);
return jsonSerDeser.load(fs.getFileSystem(), appDefPath);
}
public static Path getAppJsonPath(SliderFileSystem fs, String appName) {
@ -321,6 +347,22 @@ private static void validateApplicationResource(Resource resource,
}
}
// check if comp mem size exceeds cluster limit
public static void validateCompResourceSize(
org.apache.hadoop.yarn.api.records.Resource maxResource,
Application application) throws YarnException {
for (Component component : application.getComponents()) {
// only handle mem now.
long mem = Long.parseLong(component.getResource().getMemory());
if (mem > maxResource.getMemorySize()) {
throw new YarnException(
"Component " + component.getName() + " memory size (" + mem
+ ") is larger than configured max container memory size ("
+ maxResource.getMemorySize() + ")");
}
}
}
public static boolean hasComponent(Application application) {
if (application.getComponents() == null || application.getComponents()
.isEmpty()) {

View File

@ -21,7 +21,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import static org.apache.slider.util.ServiceApiUtil.$;
import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$;
/**
* This class defines constants that can be used in input spec for

View File

@ -22,7 +22,7 @@
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.slider.api.proto.Messages;
import org.apache.slider.common.SliderXmlConfKeys;
import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys;
import java.io.IOException;

View File

@ -226,7 +226,7 @@ public void setNumberOfRunningContainers(Long numberOfRunningContainers) {
/**
* Life time (in seconds) of the application from the time it reaches the
* STARTED state (after which it is automatically destroyed by YARN). For
* RUNNING_BUT_UNREADY state (after which it is automatically destroyed by YARN). For
* unlimited lifetime do not set a lifetime value.
**/
public Application lifetime(Long lifetime) {
@ -234,7 +234,7 @@ public Application lifetime(Long lifetime) {
return this;
}
@ApiModelProperty(example = "null", value = "Life time (in seconds) of the application from the time it reaches the STARTED state (after which it is automatically destroyed by YARN). For unlimited lifetime do not set a lifetime value.")
@ApiModelProperty(example = "null", value = "Life time (in seconds) of the application from the time it reaches the RUNNING_BUT_UNREADY state (after which it is automatically destroyed by YARN). For unlimited lifetime do not set a lifetime value.")
@JsonProperty("lifetime")
public Long getLifetime() {
return lifetime;

View File

@ -213,6 +213,9 @@ public void addContainer(Container container) {
this.containers.add(container);
}
public void removeContainer(Container container) {
containers.remove(container);
}
public Container getContainer(String id) {
for (Container container : containers) {
if (container.getId().equals(id)) {

View File

@ -21,5 +21,5 @@
* The current state of the container of an application.
**/
public enum ContainerState {
INIT, READY;
RUNNING_BUT_UNREADY, READY, STOPPED
}

View File

@ -100,6 +100,13 @@ public void setMemory(String memory) {
this.memory = memory;
}
public long getMemoryMB() {
if (this.memory == null) {
return 0;
}
return Long.valueOf(memory);
}
@Override
public boolean equals(java.lang.Object o) {
if (this == o) {

View File

@ -24,7 +24,7 @@
import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
import org.apache.hadoop.registry.client.exceptions.NoRecordException;
import org.apache.hadoop.registry.client.types.ServiceRecord;
import org.apache.slider.common.SliderKeys;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.NotFoundException;
import org.apache.slider.core.exceptions.SliderException;

View File

@ -29,7 +29,9 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusRequestProto;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.registry.client.api.RegistryConstants;
import org.apache.hadoop.registry.client.api.RegistryOperations;
import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
@ -60,9 +62,11 @@
import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.slider.api.SliderClusterProtocol;
import org.apache.slider.api.proto.Messages;
import org.apache.slider.api.resource.Application;
@ -71,17 +75,17 @@
import org.apache.slider.api.types.NodeInformationList;
import org.apache.slider.client.ipc.SliderClusterOperations;
import org.apache.slider.common.Constants;
import org.apache.slider.common.SliderExitCodes;
import org.apache.slider.common.SliderKeys;
import org.apache.slider.common.SliderXmlConfKeys;
import org.apache.slider.common.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.conf.SliderExitCodes;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.hadoop.yarn.service.conf.SliderXmlConfKeys;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
import org.apache.slider.common.params.ActionAMSuicideArgs;
import org.apache.slider.common.params.ActionClientArgs;
import org.apache.slider.common.params.ActionDependencyArgs;
import org.apache.hadoop.yarn.service.client.params.ActionDependencyArgs;
import org.apache.slider.common.params.ActionDiagnosticArgs;
import org.apache.slider.common.params.ActionExistsArgs;
import org.apache.slider.common.params.ActionFlexArgs;
import org.apache.hadoop.yarn.service.client.params.ActionFlexArgs;
import org.apache.slider.common.params.ActionFreezeArgs;
import org.apache.slider.common.params.ActionKDiagArgs;
import org.apache.slider.common.params.ActionKeytabArgs;
@ -96,9 +100,9 @@
import org.apache.slider.common.params.ActionThawArgs;
import org.apache.slider.common.params.ActionTokensArgs;
import org.apache.slider.common.params.ActionUpgradeArgs;
import org.apache.slider.common.params.Arguments;
import org.apache.slider.common.params.ClientArgs;
import org.apache.slider.common.params.CommonArgs;
import org.apache.hadoop.yarn.service.client.params.Arguments;
import org.apache.hadoop.yarn.service.client.params.ClientArgs;
import org.apache.hadoop.yarn.service.client.params.CommonArgs;
import org.apache.slider.common.tools.ConfigHelper;
import org.apache.slider.common.tools.Duration;
import org.apache.slider.common.tools.SliderFileSystem;
@ -131,12 +135,14 @@
import org.apache.slider.core.registry.retrieve.RegistryRetriever;
import org.apache.slider.core.zk.BlockingZKWatcher;
import org.apache.slider.core.zk.ZKIntegration;
import org.apache.slider.providers.AbstractClientProvider;
import org.apache.slider.providers.ProviderUtils;
import org.apache.slider.server.appmaster.SliderAppMaster;
import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
import org.apache.hadoop.yarn.service.provider.ProviderUtils;
import org.apache.slider.server.appmaster.rpc.RpcBinder;
import org.apache.hadoop.yarn.service.ClientAMProtocol;
import org.apache.hadoop.yarn.service.client.ClientAMProxy;
import org.apache.hadoop.yarn.service.ServiceMaster;
import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
import org.apache.slider.util.ServiceApiUtil;
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
@ -153,6 +159,7 @@
import java.io.OutputStreamWriter;
import java.io.PrintStream;
import java.io.PrintWriter;
import java.net.InetSocketAddress;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Arrays;
@ -170,9 +177,11 @@
import java.util.regex.Pattern;
import static org.apache.hadoop.registry.client.binding.RegistryUtils.*;
import static org.apache.hadoop.yarn.api.records.YarnApplicationState.*;
import static org.apache.slider.common.Constants.HADOOP_JAAS_DEBUG;
import static org.apache.slider.common.params.SliderActions.*;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.*;
import static org.apache.slider.common.tools.SliderUtils.*;
import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetStatusResponseProto;
/**
* Client service for Slider
@ -217,22 +226,24 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
private SliderClusterOperations sliderClusterOperations;
protected SliderFileSystem sliderFileSystem;
private YarnRPC rpc;
/**
* Yarn client service
*/
private SliderYarnClientImpl yarnClient;
private YarnAppListClient yarnAppListClient;
ResourceCalculator calculator;
/**
* The YARN registry service
*/
@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
private RegistryOperations registryOperations;
private static EnumSet<YarnApplicationState> terminatedStates = EnumSet
.of(YarnApplicationState.FINISHED, YarnApplicationState.FAILED,
YarnApplicationState.KILLED);
private static EnumSet<YarnApplicationState> terminatedStates =
EnumSet.of(FINISHED, FAILED, KILLED);
private static EnumSet<YarnApplicationState> waitingStates =
EnumSet.of(NEW, NEW_SAVING, SUBMITTED, RUNNING);
/**
* Constructor
*/
@ -277,9 +288,20 @@ protected void serviceInit(Configuration conf) throws Exception {
if (coreAction.getHadoopServicesRequired()) {
initHadoopBinding();
}
rpc = YarnRPC.create(conf);
super.serviceInit(conf);
}
@Override
protected void serviceStart() throws Exception {
super.serviceStart();
}
@Override
protected void serviceStop() throws Exception {
super.serviceStop();
}
/**
* Launched service execution. This runs {@link #exec()}
* then catches some exceptions and converts them to exit codes
@ -671,6 +693,10 @@ private ApplicationId submitApp(Application app)
YarnClientApplication yarnApp = yarnClient.createApplication();
ApplicationSubmissionContext submissionContext =
yarnApp.getApplicationSubmissionContext();
ServiceApiUtil.validateCompResourceSize(
yarnApp.getNewApplicationResponse().getMaximumResourceCapability(),
app);
applicationId = submissionContext.getApplicationId();
submissionContext.setKeepContainersAcrossApplicationAttempts(true);
if (app.getLifetime() > 0) {
@ -769,11 +795,11 @@ private String buildCommandLine(String appName, Configuration conf,
CLI.sysprop(SYSPROP_LOG4J_CONFIGURATION, LOG4J_SERVER_PROP_FILENAME);
CLI.sysprop(SYSPROP_LOG_DIR, ApplicationConstants.LOG_DIR_EXPANSION_VAR);
}
CLI.add(SliderAppMaster.SERVICE_CLASSNAME);
CLI.add(ServiceMaster.class.getCanonicalName());
CLI.add(ACTION_CREATE, appName);
//TODO debugAM CLI.add(Arguments.ARG_DEBUG)
CLI.add(Arguments.ARG_CLUSTER_URI, appRootDir.toUri());
// InetSocketAddress rmSchedulerAddress = getRmSchedulerAddress(conf);
CLI.add(Arguments.ARG_CLUSTER_URI, new Path(appRootDir, appName + ".json"));
// InetSocketAddress rmSchedulerAddress = getRmSchedulerAddress(conf);
// String rmAddr = NetUtils.getHostPortString(rmSchedulerAddress);
// CLI.add(Arguments.ARG_RM_ADDR, rmAddr);
// pass the registry binding
@ -825,12 +851,12 @@ private Map<String, String> addAMEnv(Configuration conf)
return env;
}
private Path addJarResource(String appName,
protected Path addJarResource(String appName,
Map<String, LocalResource> localResources)
throws IOException, SliderException {
Path libPath = sliderFileSystem.buildClusterDirPath(appName);
ProviderUtils
.addProviderJar(localResources, SliderAppMaster.class, SLIDER_JAR,
.addProviderJar(localResources, ServiceMaster.class, SLIDER_JAR,
sliderFileSystem, libPath, "lib", false);
Path dependencyLibTarGzip = sliderFileSystem.getDependencyTarGzip();
if (sliderFileSystem.isFile(dependencyLibTarGzip)) {
@ -1162,8 +1188,7 @@ public int actionUpdate(String clustername,
public String updateLifetime(String appName, long lifetime)
throws YarnException, IOException {
EnumSet<YarnApplicationState> appStates = EnumSet.range(
YarnApplicationState.NEW, YarnApplicationState.RUNNING);
EnumSet<YarnApplicationState> appStates = EnumSet.range(NEW, RUNNING);
ApplicationReport report = findInstance(appName, appStates);
if (report == null) {
throw new YarnException("Application not found for " + appName);
@ -1381,14 +1406,14 @@ public Set<ApplicationReport> getApplicationList(String clustername,
YarnApplicationState min, max;
if (live) {
min = YarnApplicationState.NEW;
max = YarnApplicationState.RUNNING;
min = NEW;
max = RUNNING;
} else if (!state.isEmpty()) {
YarnApplicationState stateVal = extractYarnApplicationState(state);
min = max = stateVal;
} else {
min = YarnApplicationState.NEW;
max = YarnApplicationState.KILLED;
min = NEW;
max = KILLED;
}
// get the complete list of persistent instances
Map<String, Path> persistentInstances = sliderFileSystem.listPersistentInstances();
@ -1485,7 +1510,7 @@ private YarnApplicationState extractYarnApplicationState(String state) throws
* @return true if it is running or scheduled to run.
*/
public boolean isApplicationActive(ApplicationReport report) {
return report.getYarnApplicationState() == YarnApplicationState.RUNNING
return report.getYarnApplicationState() == RUNNING
|| report.getYarnApplicationState() == YarnApplicationState.ACCEPTED;
}
@ -1547,8 +1572,7 @@ public int actionExists(String name, ActionExistsArgs args) throws YarnException
// the app exists, check that it is not in any terminated state
YarnApplicationState appstate = instance.getYarnApplicationState();
log.debug(" current app state = {}", appstate);
inDesiredState =
appstate.ordinal() < YarnApplicationState.FINISHED.ordinal();
inDesiredState = appstate.ordinal() < FINISHED.ordinal();
} else {
// scan for instance in single --state state
state = state.toUpperCase(Locale.ENGLISH);
@ -1851,6 +1875,72 @@ public Application getApplication(String appName)
return clusterOperations.getApplication();
}
private ClientAMProtocol connectToAM(String appName)
throws IOException, YarnException {
if (applicationId == null) {
Application persistedApp = ServiceApiUtil.loadApplication(sliderFileSystem,
appName);
if (persistedApp == null) {
throw new YarnException("Application " + appName
+ " doesn't exist on hdfs. Please check if the app exists in RM");
}
applicationId = ApplicationId.fromString(persistedApp.getId());
}
// Wait until app becomes running.
long startTime = System.currentTimeMillis();
int pollCount = 0;
ApplicationReport appReport = null;
while (true) {
appReport = yarnClient.getApplicationReport(applicationId);
YarnApplicationState state = appReport.getYarnApplicationState();
if (state == RUNNING) {
break;
}
if (terminatedStates.contains(state)) {
throw new YarnException(
"Failed to getStatus " + applicationId + ": " + appReport
.getDiagnostics());
}
long elapsedMillis = System.currentTimeMillis() - startTime;
// if over 5 min, quit
if (elapsedMillis >= 300000) {
throw new YarnException(
"Timed out while waiting for application " + applicationId
+ " to be running");
}
if (++pollCount % 10 == 0) {
log.info("Waiting for application {} to be running, current state is {}",
applicationId, state);
}
try {
Thread.sleep(3000);
} catch (InterruptedException ie) {
String msg =
"Interrupted while waiting for application " + applicationId
+ " to be running.";
throw new YarnException(msg, ie);
}
}
// Make the connection
InetSocketAddress address = NetUtils
.createSocketAddrForHost(appReport.getHost(), appReport.getRpcPort());
return ClientAMProxy
.createProxy(getConfig(), ClientAMProtocol.class,
UserGroupInformation.getCurrentUser(), rpc, address);
}
public Application getStatus(String appName)
throws IOException, YarnException {
ClientAMProtocol proxy = connectToAM(appName);
GetStatusResponseProto response =
proxy.getStatus(GetStatusRequestProto.newBuilder().build());
Application app = jsonSerDeser.fromJson(response.getStatus());
return app;
}
/**
* Bond to a running cluster
* @param clustername cluster name
@ -2686,8 +2776,7 @@ private int actionTokens(ActionTokensArgs args)
@VisibleForTesting
public ApplicationReport monitorAppToRunning(Duration duration)
throws YarnException, IOException {
return yarnClient.monitorAppToState(applicationId, YarnApplicationState
.RUNNING, duration);
return yarnClient.monitorAppToState(applicationId, RUNNING, duration);
}
}

View File

@ -26,9 +26,9 @@
import org.apache.slider.common.params.AbstractClusterBuildingActionArgs;
import org.apache.slider.common.params.ActionAMSuicideArgs;
import org.apache.slider.common.params.ActionClientArgs;
import org.apache.slider.common.params.ActionDependencyArgs;
import org.apache.hadoop.yarn.service.client.params.ActionDependencyArgs;
import org.apache.slider.common.params.ActionDiagnosticArgs;
import org.apache.slider.common.params.ActionFlexArgs;
import org.apache.hadoop.yarn.service.client.params.ActionFlexArgs;
import org.apache.slider.common.params.ActionFreezeArgs;
import org.apache.slider.common.params.ActionKeytabArgs;
import org.apache.slider.common.params.ActionNodesArgs;

View File

@ -20,44 +20,29 @@
import com.google.common.base.Preconditions;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records;
import org.apache.slider.api.types.NodeInformation;
import org.apache.slider.api.types.NodeInformationList;
import org.apache.slider.common.SliderKeys;
import org.apache.slider.common.params.ActionNodesArgs;
import org.apache.slider.common.tools.CoreFileSystem;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.slider.common.tools.Duration;
import org.apache.slider.common.tools.SliderFileSystem;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.BindException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
//TODO, Remove this class and YarnAppListClient

View File

@ -18,6 +18,8 @@
package org.apache.slider.common.params;
import org.apache.hadoop.yarn.service.client.params.ArgOps;
import org.apache.hadoop.yarn.service.client.params.Arguments;
/**
* Base class for all the delegates

View File

@ -21,6 +21,8 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.ParametersDelegate;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.ComponentArgsDelegate;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import java.io.File;

View File

@ -20,8 +20,10 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_AM_SUICIDE},
@Parameters(commandNames = { SliderActions.ACTION_AM_SUICIDE},
commandDescription = SliderActions.DESCRIBE_ACTION_AM_SUICIDE)
public class ActionAMSuicideArgs extends AbstractActionArgs {

View File

@ -20,10 +20,12 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import java.io.File;
@Parameters(commandNames = {SliderActions.ACTION_CLIENT},
@Parameters(commandNames = { SliderActions.ACTION_CLIENT},
commandDescription = SliderActions.DESCRIBE_ACTION_CLIENT)
public class ActionClientArgs extends AbstractActionArgs {

View File

@ -19,9 +19,11 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(
commandNames = {SliderActions.ACTION_DIAGNOSTICS},
commandNames = { SliderActions.ACTION_DIAGNOSTICS},
commandDescription = SliderActions.DESCRIBE_ACTION_DIAGNOSTIC)
public class ActionDiagnosticArgs extends AbstractActionArgs {

View File

@ -20,10 +20,12 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import java.io.File;
@Parameters(commandNames = {SliderActions.ACTION_EXISTS},
@Parameters(commandNames = { SliderActions.ACTION_EXISTS},
commandDescription = SliderActions.DESCRIBE_ACTION_EXISTS)
public class ActionExistsArgs extends AbstractActionArgs {

View File

@ -21,8 +21,10 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.beust.jcommander.ParametersDelegate;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_STOP },
@Parameters(commandNames = { SliderActions.ACTION_STOP },
commandDescription = SliderActions.DESCRIBE_ACTION_FREEZE)
public class ActionFreezeArgs extends AbstractActionArgs implements

View File

@ -19,11 +19,13 @@
package org.apache.slider.common.params;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
/**
* The Help command
*/
@Parameters(commandNames = {SliderActions.ACTION_HELP},
@Parameters(commandNames = { SliderActions.ACTION_HELP},
commandDescription = SliderActions.DESCRIBE_ACTION_HELP)
public class ActionHelpArgs extends AbstractActionArgs {
@Override

View File

@ -20,6 +20,8 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.slider.common.tools.SliderUtils;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.UsageException;
@ -28,7 +30,7 @@
import java.util.ArrayList;
import java.util.List;
@Parameters(commandNames = {SliderActions.ACTION_KDIAG},
@Parameters(commandNames = { SliderActions.ACTION_KDIAG},
commandDescription = SliderActions.DESCRIBE_ACTION_KDIAG)
public class ActionKDiagArgs extends AbstractActionArgs {

View File

@ -20,8 +20,10 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_KEYTAB},
@Parameters(commandNames = { SliderActions.ACTION_KEYTAB},
commandDescription = SliderActions.DESCRIBE_ACTION_KEYTAB)
public class ActionKeytabArgs extends AbstractActionArgs {

View File

@ -20,8 +20,10 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_KILL_CONTAINER},
@Parameters(commandNames = { SliderActions.ACTION_KILL_CONTAINER},
commandDescription = SliderActions.DESCRIBE_ACTION_KILL_CONTAINER)
public class ActionKillContainerArgs extends AbstractActionArgs {

View File

@ -23,8 +23,10 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_LIST},
@Parameters(commandNames = { SliderActions.ACTION_LIST},
commandDescription = SliderActions.DESCRIBE_ACTION_LIST)
public class ActionListArgs extends AbstractActionArgs {

View File

@ -21,12 +21,14 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.UsageException;
import java.io.File;
@Parameters(commandNames = {SliderActions.ACTION_LOOKUP},
@Parameters(commandNames = { SliderActions.ACTION_LOOKUP},
commandDescription = SliderActions.DESCRIBE_ACTION_LOOKUP)
public class ActionLookupArgs extends AbstractActionArgs {

View File

@ -20,10 +20,12 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import java.io.File;
@Parameters(commandNames = {SliderActions.ACTION_NODES},
@Parameters(commandNames = { SliderActions.ACTION_NODES},
commandDescription = SliderActions.DESCRIBE_ACTION_NODES)
public class ActionNodesArgs extends AbstractActionArgs {

View File

@ -20,13 +20,16 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.slider.common.SliderKeys;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.Arguments;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.hadoop.yarn.service.conf.SliderKeys;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.UsageException;
import org.apache.slider.core.registry.docstore.ConfigFormat;
import static org.apache.slider.common.params.SliderActions.ACTION_REGISTRY;
import static org.apache.slider.common.params.SliderActions.DESCRIBE_ACTION_REGISTRY;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_REGISTRY;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.DESCRIBE_ACTION_REGISTRY;
import java.io.File;
/**

View File

@ -21,13 +21,15 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.UsageException;
import java.io.File;
import static org.apache.slider.common.params.SliderActions.ACTION_RESOLVE;
import static org.apache.slider.common.params.SliderActions.DESCRIBE_ACTION_REGISTRY;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.ACTION_RESOLVE;
import static org.apache.hadoop.yarn.service.client.params.SliderActions.DESCRIBE_ACTION_REGISTRY;
/**
* Resolve registry entries

View File

@ -19,8 +19,10 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_RESOURCE},
@Parameters(commandNames = { SliderActions.ACTION_RESOURCE},
commandDescription = SliderActions.DESCRIBE_ACTION_RESOURCE)
public class ActionResourceArgs extends AbstractActionArgs {

View File

@ -20,8 +20,10 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_STATUS},
@Parameters(commandNames = { SliderActions.ACTION_STATUS},
commandDescription = SliderActions.DESCRIBE_ACTION_STATUS)
public class ActionStatusArgs extends AbstractActionArgs {

View File

@ -21,10 +21,12 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.beust.jcommander.ParametersDelegate;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import java.io.File;
@Parameters(commandNames = {SliderActions.ACTION_START },
@Parameters(commandNames = { SliderActions.ACTION_START },
commandDescription = SliderActions.DESCRIBE_ACTION_THAW)
public class ActionThawArgs extends AbstractActionArgs implements
WaitTimeAccessor,

View File

@ -20,12 +20,14 @@
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.AbstractActionArgs;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
import org.apache.slider.core.exceptions.BadCommandArgumentsException;
import org.apache.slider.core.exceptions.UsageException;
import java.io.File;
@Parameters(commandNames = {SliderActions.ACTION_TOKENS},
@Parameters(commandNames = { SliderActions.ACTION_TOKENS},
commandDescription = "save tokens to a file or list tokens in a file")
public class ActionTokensArgs extends AbstractActionArgs {

View File

@ -19,8 +19,9 @@
package org.apache.slider.common.params;
import com.beust.jcommander.Parameters;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = {SliderActions.ACTION_UPDATE},
@Parameters(commandNames = { SliderActions.ACTION_UPDATE},
commandDescription = SliderActions.DESCRIBE_ACTION_UPDATE)
public class ActionUpdateArgs extends AbstractClusterBuildingActionArgs {

View File

@ -18,11 +18,8 @@
package org.apache.slider.common.params;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.yarn.service.client.params.SliderActions;
@Parameters(commandNames = { SliderActions.ACTION_UPGRADE },
commandDescription = SliderActions.DESCRIBE_ACTION_UPGRADE)

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