|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.service.client;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.commons.lang.StringUtils;
|
|
|
import org.apache.curator.framework.CuratorFramework;
|
|
|
import org.apache.curator.framework.CuratorFrameworkFactory;
|
|
@@ -55,7 +56,9 @@ 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.RestartServiceRequestProto;
|
|
|
import org.apache.hadoop.yarn.proto.ClientAMProtocol.StopRequestProto;
|
|
|
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.UpgradeServiceRequestProto;
|
|
|
import org.apache.hadoop.yarn.service.ClientAMProtocol;
|
|
|
import org.apache.hadoop.yarn.service.ServiceMaster;
|
|
|
import org.apache.hadoop.yarn.service.api.records.Component;
|
|
@@ -73,8 +76,8 @@ import org.apache.hadoop.yarn.service.provider.AbstractClientProvider;
|
|
|
import org.apache.hadoop.yarn.service.provider.ProviderUtils;
|
|
|
import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
|
|
|
import org.apache.hadoop.yarn.service.utils.ServiceRegistryUtils;
|
|
|
-import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
|
|
|
import org.apache.hadoop.yarn.service.utils.ServiceUtils;
|
|
|
+import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
|
|
|
import org.apache.hadoop.yarn.service.utils.ZookeeperUtils;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.apache.hadoop.yarn.util.Times;
|
|
@@ -186,6 +189,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
return service;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
public int actionSave(String fileName, String serviceName, Long lifetime,
|
|
|
String queue) throws IOException, YarnException {
|
|
|
return actionBuild(loadAppJsonFromLocalFS(fileName, serviceName,
|
|
@@ -194,9 +198,54 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
|
|
|
public int actionBuild(Service service)
|
|
|
throws YarnException, IOException {
|
|
|
- Path appDir = checkAppNotExistOnHdfs(service);
|
|
|
ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
|
|
|
- createDirAndPersistApp(appDir, service);
|
|
|
+ Path appDir = checkAppNotExistOnHdfs(service, false);
|
|
|
+ ServiceApiUtil.createDirAndPersistApp(fs, appDir, service);
|
|
|
+ return EXIT_SUCCESS;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int actionUpgrade(String appName, String fileName)
|
|
|
+ throws IOException, YarnException {
|
|
|
+ checkAppExistOnHdfs(appName);
|
|
|
+ Service upgradeService = loadAppJsonFromLocalFS(fileName, appName,
|
|
|
+ null, null);
|
|
|
+ return actionUpgrade(upgradeService);
|
|
|
+ }
|
|
|
+
|
|
|
+ public int actionUpgrade(Service service) throws YarnException, IOException {
|
|
|
+ Service persistedService =
|
|
|
+ ServiceApiUtil.loadService(fs, service.getName());
|
|
|
+ if (!StringUtils.isEmpty(persistedService.getId())) {
|
|
|
+ cachedAppInfo.put(persistedService.getName(), new AppInfo(
|
|
|
+ ApplicationId.fromString(persistedService.getId()),
|
|
|
+ persistedService.getKerberosPrincipal().getPrincipalName()));
|
|
|
+ }
|
|
|
+
|
|
|
+ if (persistedService.getVersion().equals(service.getVersion())) {
|
|
|
+ String message =
|
|
|
+ service.getName() + " is already at version " + service.getVersion()
|
|
|
+ + ". There is nothing to upgrade.";
|
|
|
+ LOG.error(message);
|
|
|
+ throw new YarnException(message);
|
|
|
+ }
|
|
|
+
|
|
|
+ Path serviceUpgradeDir = checkAppNotExistOnHdfs(service, true);
|
|
|
+ ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
|
|
|
+ ServiceApiUtil.createDirAndPersistApp(fs, serviceUpgradeDir, service);
|
|
|
+
|
|
|
+ ApplicationReport appReport =
|
|
|
+ yarnClient.getApplicationReport(getAppId(service.getName()));
|
|
|
+ if (StringUtils.isEmpty(appReport.getHost())) {
|
|
|
+ throw new YarnException(service.getName() + " AM hostname is empty");
|
|
|
+ }
|
|
|
+ ClientAMProtocol proxy = createAMProxy(service.getName(), appReport);
|
|
|
+
|
|
|
+ UpgradeServiceRequestProto.Builder requestBuilder =
|
|
|
+ UpgradeServiceRequestProto.newBuilder();
|
|
|
+ requestBuilder.setVersion(service.getVersion());
|
|
|
+
|
|
|
+ proxy.upgrade(requestBuilder.build());
|
|
|
return EXIT_SUCCESS;
|
|
|
}
|
|
|
|
|
@@ -212,16 +261,16 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
String serviceName = service.getName();
|
|
|
ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
|
|
|
verifyNoLiveAppInRM(serviceName, "create");
|
|
|
- Path appDir = checkAppNotExistOnHdfs(service);
|
|
|
+ Path appDir = checkAppNotExistOnHdfs(service, false);
|
|
|
|
|
|
// Write the definition first and then submit - AM will read the definition
|
|
|
- createDirAndPersistApp(appDir, service);
|
|
|
+ ServiceApiUtil.createDirAndPersistApp(fs, appDir, service);
|
|
|
ApplicationId appId = submitApp(service);
|
|
|
cachedAppInfo.put(serviceName, new AppInfo(appId, service
|
|
|
.getKerberosPrincipal().getPrincipalName()));
|
|
|
service.setId(appId.toString());
|
|
|
// update app definition with appId
|
|
|
- persistAppDef(appDir, service);
|
|
|
+ ServiceApiUtil.writeAppDefinition(fs, appDir, service);
|
|
|
return appId;
|
|
|
}
|
|
|
|
|
@@ -349,6 +398,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
return original;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
public int actionStop(String serviceName)
|
|
|
throws YarnException, IOException {
|
|
|
return actionStop(serviceName, true);
|
|
@@ -424,6 +474,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
return EXIT_SUCCESS;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
public int actionDestroy(String serviceName) throws YarnException,
|
|
|
IOException {
|
|
|
ServiceApiUtil.validateNameFormat(serviceName, getConfig());
|
|
@@ -557,8 +608,8 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private ApplicationId submitApp(Service app)
|
|
|
- throws IOException, YarnException {
|
|
|
+ @VisibleForTesting
|
|
|
+ ApplicationId submitApp(Service app) throws IOException, YarnException {
|
|
|
String serviceName = app.getName();
|
|
|
Configuration conf = getConfig();
|
|
|
Path appRootDir = fs.buildClusterDirPath(app.getName());
|
|
@@ -772,29 +823,64 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
return hasAMLog4j;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
public int actionStart(String serviceName) throws YarnException, IOException {
|
|
|
ServiceApiUtil.validateNameFormat(serviceName, getConfig());
|
|
|
- Path appDir = checkAppExistOnHdfs(serviceName);
|
|
|
- Service service = ServiceApiUtil.loadService(fs, serviceName);
|
|
|
- ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
|
|
|
- // see if it is actually running and bail out;
|
|
|
- verifyNoLiveAppInRM(serviceName, "start");
|
|
|
- ApplicationId appId = submitApp(service);
|
|
|
- service.setId(appId.toString());
|
|
|
- // write app definition on to hdfs
|
|
|
- Path appJson = persistAppDef(appDir, service);
|
|
|
- LOG.info("Persisted service " + service.getName() + " at " + appJson);
|
|
|
- return 0;
|
|
|
+ Service liveService = getStatus(serviceName);
|
|
|
+ if (liveService == null ||
|
|
|
+ !liveService.getState().equals(ServiceState.UPGRADING)) {
|
|
|
+ Path appDir = checkAppExistOnHdfs(serviceName);
|
|
|
+ Service service = ServiceApiUtil.loadService(fs, serviceName);
|
|
|
+ ServiceApiUtil.validateAndResolveService(service, fs, getConfig());
|
|
|
+ // see if it is actually running and bail out;
|
|
|
+ verifyNoLiveAppInRM(serviceName, "start");
|
|
|
+ ApplicationId appId = submitApp(service);
|
|
|
+ service.setId(appId.toString());
|
|
|
+ // write app definition on to hdfs
|
|
|
+ Path appJson = ServiceApiUtil.writeAppDefinition(fs, appDir, service);
|
|
|
+ LOG.info("Persisted service " + service.getName() + " at " + appJson);
|
|
|
+ return 0;
|
|
|
+ } else {
|
|
|
+ LOG.info("Finalize service {} upgrade");
|
|
|
+ ApplicationReport appReport =
|
|
|
+ yarnClient.getApplicationReport(getAppId(serviceName));
|
|
|
+ if (StringUtils.isEmpty(appReport.getHost())) {
|
|
|
+ throw new YarnException(serviceName + " AM hostname is empty");
|
|
|
+ }
|
|
|
+ ClientAMProtocol proxy = createAMProxy(serviceName, appReport);
|
|
|
+
|
|
|
+ RestartServiceRequestProto.Builder requestBuilder =
|
|
|
+ RestartServiceRequestProto.newBuilder();
|
|
|
+ proxy.restart(requestBuilder.build());
|
|
|
+ return 0;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- private Path checkAppNotExistOnHdfs(Service service)
|
|
|
+ /**
|
|
|
+ * Verifies that the service definition does not exist on hdfs.
|
|
|
+ *
|
|
|
+ * @param service service
|
|
|
+ * @param isUpgrade true for upgrades; false otherwise
|
|
|
+ * @return path to the service definition..
|
|
|
+ * @throws IOException
|
|
|
+ * @throws SliderException
|
|
|
+ */
|
|
|
+ private Path checkAppNotExistOnHdfs(Service service, boolean isUpgrade)
|
|
|
throws IOException, SliderException {
|
|
|
- Path appDir = fs.buildClusterDirPath(service.getName());
|
|
|
+ Path appDir = !isUpgrade ? fs.buildClusterDirPath(service.getName()) :
|
|
|
+ fs.buildClusterUpgradeDirPath(service.getName(), service.getVersion());
|
|
|
fs.verifyDirectoryNonexistent(
|
|
|
new Path(appDir, service.getName() + ".json"));
|
|
|
return appDir;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Verifies that the service exists on hdfs.
|
|
|
+ * @param serviceName service name
|
|
|
+ * @return path to the service definition.
|
|
|
+ * @throws IOException
|
|
|
+ * @throws SliderException
|
|
|
+ */
|
|
|
private Path checkAppExistOnHdfs(String serviceName)
|
|
|
throws IOException, SliderException {
|
|
|
Path appDir = fs.buildClusterDirPath(serviceName);
|
|
@@ -802,20 +888,6 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
return appDir;
|
|
|
}
|
|
|
|
|
|
- private void createDirAndPersistApp(Path appDir, Service service)
|
|
|
- throws IOException, SliderException {
|
|
|
- FsPermission appDirPermission = new FsPermission("750");
|
|
|
- fs.createWithPermissions(appDir, appDirPermission);
|
|
|
- Path appJson = persistAppDef(appDir, service);
|
|
|
- LOG.info("Persisted service " + service.getName() + " at " + appJson);
|
|
|
- }
|
|
|
-
|
|
|
- private Path persistAppDef(Path appDir, Service service) throws IOException {
|
|
|
- Path appJson = new Path(appDir, service.getName() + ".json");
|
|
|
- jsonSerDeser.save(fs.getFileSystem(), appJson, service, true);
|
|
|
- return appJson;
|
|
|
- }
|
|
|
-
|
|
|
private void addHdfsDelegationTokenIfSecure(ContainerLaunchContext amContext)
|
|
|
throws IOException {
|
|
|
if (!UserGroupInformation.isSecurityEnabled()) {
|
|
@@ -1074,6 +1146,17 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
|
|
|
UserGroupInformation.getCurrentUser(), rpc, address);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ void setFileSystem(SliderFileSystem fileSystem)
|
|
|
+ throws IOException {
|
|
|
+ this.fs = fileSystem;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ void setYarnClient(YarnClient yarnClient) {
|
|
|
+ this.yarnClient = yarnClient;
|
|
|
+ }
|
|
|
+
|
|
|
public synchronized ApplicationId getAppId(String serviceName)
|
|
|
throws IOException, YarnException {
|
|
|
if (cachedAppInfo.containsKey(serviceName)) {
|