|
@@ -75,6 +75,7 @@ import org.apache.slider.providers.ProviderRole;
|
|
import org.apache.slider.providers.ProviderUtils;
|
|
import org.apache.slider.providers.ProviderUtils;
|
|
import org.apache.slider.providers.agent.application.metadata.AbstractComponent;
|
|
import org.apache.slider.providers.agent.application.metadata.AbstractComponent;
|
|
import org.apache.slider.providers.agent.application.metadata.Application;
|
|
import org.apache.slider.providers.agent.application.metadata.Application;
|
|
|
|
+import org.apache.slider.providers.agent.application.metadata.CommandOrder;
|
|
import org.apache.slider.providers.agent.application.metadata.CommandScript;
|
|
import org.apache.slider.providers.agent.application.metadata.CommandScript;
|
|
import org.apache.slider.providers.agent.application.metadata.Component;
|
|
import org.apache.slider.providers.agent.application.metadata.Component;
|
|
import org.apache.slider.providers.agent.application.metadata.ComponentCommand;
|
|
import org.apache.slider.providers.agent.application.metadata.ComponentCommand;
|
|
@@ -128,6 +129,7 @@ import java.util.LinkedHashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Locale;
|
|
import java.util.Locale;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.Map.Entry;
|
|
import java.util.Scanner;
|
|
import java.util.Scanner;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.TreeMap;
|
|
import java.util.TreeMap;
|
|
@@ -135,7 +137,9 @@ import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.CopyOnWriteArrayList;
|
|
import java.util.concurrent.CopyOnWriteArrayList;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
+import java.util.regex.Pattern;
|
|
|
|
|
|
|
|
+import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
|
|
import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_AGENTS;
|
|
import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_AGENTS;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -170,10 +174,10 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
private int heartbeatMonitorInterval = 0;
|
|
private int heartbeatMonitorInterval = 0;
|
|
private AgentClientProvider clientProvider;
|
|
private AgentClientProvider clientProvider;
|
|
private AtomicInteger taskId = new AtomicInteger(0);
|
|
private AtomicInteger taskId = new AtomicInteger(0);
|
|
- private volatile Metainfo metaInfo = null;
|
|
|
|
|
|
+ private volatile Map<String, MetainfoHolder> metaInfoMap = new HashMap<>();
|
|
private SliderFileSystem fileSystem = null;
|
|
private SliderFileSystem fileSystem = null;
|
|
private Map<String, DefaultConfig> defaultConfigs = null;
|
|
private Map<String, DefaultConfig> defaultConfigs = null;
|
|
- private ComponentCommandOrder commandOrder = null;
|
|
|
|
|
|
+ private ComponentCommandOrder commandOrder = new ComponentCommandOrder();
|
|
private HeartbeatMonitor monitor;
|
|
private HeartbeatMonitor monitor;
|
|
private Boolean canAnyMasterPublish = null;
|
|
private Boolean canAnyMasterPublish = null;
|
|
private AgentLaunchParameter agentLaunchParameter = null;
|
|
private AgentLaunchParameter agentLaunchParameter = null;
|
|
@@ -208,6 +212,17 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
private final Map<String, Set<String>> containerExportsMap =
|
|
private final Map<String, Set<String>> containerExportsMap =
|
|
new HashMap<String, Set<String>>();
|
|
new HashMap<String, Set<String>>();
|
|
|
|
|
|
|
|
+ private static class MetainfoHolder {
|
|
|
|
+ Metainfo metaInfo;
|
|
|
|
+ private Map<String, DefaultConfig> defaultConfigs = null;
|
|
|
|
+
|
|
|
|
+ public MetainfoHolder(Metainfo metaInfo,
|
|
|
|
+ Map<String, DefaultConfig> defaultConfigs) {
|
|
|
|
+ this.metaInfo = metaInfo;
|
|
|
|
+ this.defaultConfigs = defaultConfigs;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create an instance of AgentProviderService
|
|
* Create an instance of AgentProviderService
|
|
*/
|
|
*/
|
|
@@ -252,10 +267,11 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
Set<String> names = resources.getComponentNames();
|
|
Set<String> names = resources.getComponentNames();
|
|
names.remove(SliderKeys.COMPONENT_AM);
|
|
names.remove(SliderKeys.COMPONENT_AM);
|
|
for (String name : names) {
|
|
for (String name : names) {
|
|
- Component componentDef = getMetaInfo().getApplicationComponent(name);
|
|
|
|
|
|
+ Component componentDef = getApplicationComponent(name);
|
|
if (componentDef == null) {
|
|
if (componentDef == null) {
|
|
- throw new BadConfigException(
|
|
|
|
- "Component %s is not a member of application.", name);
|
|
|
|
|
|
+ // component member is validated elsewhere, so we don't need to throw
|
|
|
|
+ // an exception here
|
|
|
|
+ continue;
|
|
}
|
|
}
|
|
|
|
|
|
MapOperations componentConfig = resources.getMandatoryComponent(name);
|
|
MapOperations componentConfig = resources.getMandatoryComponent(name);
|
|
@@ -277,32 +293,67 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
|
|
|
|
// Reads the metainfo.xml in the application package and loads it
|
|
// Reads the metainfo.xml in the application package and loads it
|
|
private void buildMetainfo(AggregateConf instanceDefinition,
|
|
private void buildMetainfo(AggregateConf instanceDefinition,
|
|
- SliderFileSystem fileSystem) throws IOException, SliderException {
|
|
|
|
- String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
|
|
|
|
- .getAppConfOperations());
|
|
|
|
|
|
+ SliderFileSystem fileSystem,
|
|
|
|
+ String roleGroup)
|
|
|
|
+ throws IOException, SliderException {
|
|
|
|
+ String mapKey = instanceDefinition.getAppConfOperations()
|
|
|
|
+ .getComponentOpt(roleGroup, ROLE_PREFIX, DEFAULT_METAINFO_MAP_KEY);
|
|
|
|
+ String appDef = SliderUtils.getApplicationDefinitionPath(
|
|
|
|
+ instanceDefinition.getAppConfOperations(), roleGroup);
|
|
|
|
+ MapOperations component = null;
|
|
|
|
+ if (roleGroup != null) {
|
|
|
|
+ component = instanceDefinition.getAppConfOperations().getComponent(roleGroup);
|
|
|
|
+ }
|
|
|
|
|
|
- if (metaInfo == null) {
|
|
|
|
|
|
+ MetainfoHolder metaInfoHolder = metaInfoMap.get(mapKey);
|
|
|
|
+ if (metaInfoHolder == null) {
|
|
synchronized (syncLock) {
|
|
synchronized (syncLock) {
|
|
- if (metaInfo == null) {
|
|
|
|
|
|
+ if (this.fileSystem == null) {
|
|
this.fileSystem = fileSystem;
|
|
this.fileSystem = fileSystem;
|
|
|
|
+ }
|
|
|
|
+ metaInfoHolder = metaInfoMap.get(mapKey);
|
|
|
|
+ if (metaInfoHolder == null) {
|
|
readAndSetHeartbeatMonitoringInterval(instanceDefinition);
|
|
readAndSetHeartbeatMonitoringInterval(instanceDefinition);
|
|
initializeAgentDebugCommands(instanceDefinition);
|
|
initializeAgentDebugCommands(instanceDefinition);
|
|
|
|
|
|
- metaInfo = getApplicationMetainfo(fileSystem, appDef, false);
|
|
|
|
|
|
+ Metainfo metaInfo = getApplicationMetainfo(fileSystem, appDef, false);
|
|
log.info("Master package metainfo: {}", metaInfo.toString());
|
|
log.info("Master package metainfo: {}", metaInfo.toString());
|
|
if (metaInfo == null || metaInfo.getApplication() == null) {
|
|
if (metaInfo == null || metaInfo.getApplication() == null) {
|
|
log.error("metainfo.xml is unavailable or malformed at {}.", appDef);
|
|
log.error("metainfo.xml is unavailable or malformed at {}.", appDef);
|
|
throw new SliderException(
|
|
throw new SliderException(
|
|
"metainfo.xml is required in app package.");
|
|
"metainfo.xml is required in app package.");
|
|
}
|
|
}
|
|
- commandOrder = new ComponentCommandOrder(metaInfo.getApplication().getCommandOrders());
|
|
|
|
- defaultConfigs = initializeDefaultConfigs(fileSystem, appDef, metaInfo);
|
|
|
|
|
|
+ List<CommandOrder> commandOrders = metaInfo.getApplication()
|
|
|
|
+ .getCommandOrders();
|
|
|
|
+ if (!DEFAULT_METAINFO_MAP_KEY.equals(mapKey)) {
|
|
|
|
+ for (Component comp : metaInfo.getApplication().getComponents()) {
|
|
|
|
+ comp.setName(mapKey + comp.getName());
|
|
|
|
+ log.info("Modifying external metainfo component name to {}",
|
|
|
|
+ comp.getName());
|
|
|
|
+ }
|
|
|
|
+ for (CommandOrder co : commandOrders) {
|
|
|
|
+ log.info("Adding prefix {} to command order {}",
|
|
|
|
+ mapKey, co);
|
|
|
|
+ co.setCommand(mapKey + co.getCommand());
|
|
|
|
+ co.setRequires(mapKey + co.getRequires());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ log.debug("Merging command orders {} for {}", commandOrders,
|
|
|
|
+ roleGroup);
|
|
|
|
+ commandOrder.mergeCommandOrders(commandOrders,
|
|
|
|
+ instanceDefinition.getResourceOperations());
|
|
|
|
+ Map<String, DefaultConfig> defaultConfigs =
|
|
|
|
+ initializeDefaultConfigs(fileSystem, appDef, metaInfo);
|
|
|
|
+ metaInfoMap.put(mapKey, new MetainfoHolder(metaInfo, defaultConfigs));
|
|
monitor = new HeartbeatMonitor(this, getHeartbeatMonitorInterval());
|
|
monitor = new HeartbeatMonitor(this, getHeartbeatMonitorInterval());
|
|
monitor.start();
|
|
monitor.start();
|
|
|
|
|
|
// build a map from component to metainfo
|
|
// build a map from component to metainfo
|
|
String addonAppDefString = instanceDefinition.getAppConfOperations()
|
|
String addonAppDefString = instanceDefinition.getAppConfOperations()
|
|
.getGlobalOptions().getOption(AgentKeys.ADDONS, null);
|
|
.getGlobalOptions().getOption(AgentKeys.ADDONS, null);
|
|
|
|
+ if (component != null) {
|
|
|
|
+ addonAppDefString = component.getOption(AgentKeys.ADDONS, addonAppDefString);
|
|
|
|
+ }
|
|
log.debug("All addon appdefs: {}", addonAppDefString);
|
|
log.debug("All addon appdefs: {}", addonAppDefString);
|
|
if (addonAppDefString != null) {
|
|
if (addonAppDefString != null) {
|
|
Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
|
|
Scanner scanner = new Scanner(addonAppDefString).useDelimiter(",");
|
|
@@ -310,6 +361,9 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
String addonAppDef = scanner.next();
|
|
String addonAppDef = scanner.next();
|
|
String addonAppDefPath = instanceDefinition
|
|
String addonAppDefPath = instanceDefinition
|
|
.getAppConfOperations().getGlobalOptions().get(addonAppDef);
|
|
.getAppConfOperations().getGlobalOptions().get(addonAppDef);
|
|
|
|
+ if (component != null) {
|
|
|
|
+ addonAppDefPath = component.getOption(addonAppDef, addonAppDefPath);
|
|
|
|
+ }
|
|
log.debug("Addon package {} is stored at: {}", addonAppDef
|
|
log.debug("Addon package {} is stored at: {}", addonAppDef
|
|
+ addonAppDefPath);
|
|
+ addonAppDefPath);
|
|
Metainfo addonMetaInfo = getApplicationMetainfo(fileSystem,
|
|
Metainfo addonMetaInfo = getApplicationMetainfo(fileSystem,
|
|
@@ -328,9 +382,10 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void initializeApplicationConfiguration(
|
|
public void initializeApplicationConfiguration(
|
|
- AggregateConf instanceDefinition, SliderFileSystem fileSystem)
|
|
|
|
|
|
+ AggregateConf instanceDefinition, SliderFileSystem fileSystem,
|
|
|
|
+ String roleGroup)
|
|
throws IOException, SliderException {
|
|
throws IOException, SliderException {
|
|
- buildMetainfo(instanceDefinition, fileSystem);
|
|
|
|
|
|
+ buildMetainfo(instanceDefinition, fileSystem, roleGroup);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -349,9 +404,9 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
String roleName = providerRole.name;
|
|
String roleName = providerRole.name;
|
|
String roleGroup = providerRole.group;
|
|
String roleGroup = providerRole.group;
|
|
String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
|
|
String appDef = SliderUtils.getApplicationDefinitionPath(instanceDefinition
|
|
- .getAppConfOperations());
|
|
|
|
|
|
+ .getAppConfOperations(), roleGroup);
|
|
|
|
|
|
- initializeApplicationConfiguration(instanceDefinition, fileSystem);
|
|
|
|
|
|
+ initializeApplicationConfiguration(instanceDefinition, fileSystem, roleGroup);
|
|
|
|
|
|
log.info("Build launch context for Agent");
|
|
log.info("Build launch context for Agent");
|
|
log.debug(instanceDefinition.toString());
|
|
log.debug(instanceDefinition.toString());
|
|
@@ -439,7 +494,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
LocalResourceType.ARCHIVE);
|
|
LocalResourceType.ARCHIVE);
|
|
launcher.addLocalResource(AgentKeys.APP_DEFINITION_DIR, appDefRes);
|
|
launcher.addLocalResource(AgentKeys.APP_DEFINITION_DIR, appDefRes);
|
|
|
|
|
|
- for (Package pkg : getMetaInfo().getApplication().getPackages()) {
|
|
|
|
|
|
+ for (Package pkg : getMetaInfo(roleGroup).getApplication().getPackages()) {
|
|
Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
|
|
Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
|
|
if (!fileSystem.isFile(pkgPath)) {
|
|
if (!fileSystem.isFile(pkgPath)) {
|
|
pkgPath = fileSystem.buildResourcePath(getClusterName(),
|
|
pkgPath = fileSystem.buildResourcePath(getClusterName(),
|
|
@@ -505,7 +560,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
Map<String, Map<String, String>> configurations =
|
|
Map<String, Map<String, String>> configurations =
|
|
buildCommandConfigurations(instanceDefinition.getAppConfOperations(),
|
|
buildCommandConfigurations(instanceDefinition.getAppConfOperations(),
|
|
container.getId().toString(), roleName, roleGroup);
|
|
container.getId().toString(), roleName, roleGroup);
|
|
- localizeConfigFiles(launcher, roleName, roleGroup, getMetaInfo(),
|
|
|
|
|
|
+ localizeConfigFiles(launcher, roleName, roleGroup, getMetaInfo(roleGroup),
|
|
configurations, launcher.getEnv(), fileSystem);
|
|
configurations, launcher.getEnv(), fileSystem);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -597,7 +652,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
// initialize the component instance state
|
|
// initialize the component instance state
|
|
getComponentStatuses().put(label,
|
|
getComponentStatuses().put(label,
|
|
new ComponentInstanceState(
|
|
new ComponentInstanceState(
|
|
- roleName,
|
|
|
|
|
|
+ roleGroup,
|
|
container.getId(),
|
|
container.getId(),
|
|
getClusterInfoPropertyValue(OptionKeys.APPLICATION_NAME),
|
|
getClusterInfoPropertyValue(OptionKeys.APPLICATION_NAME),
|
|
pkgStatuses));
|
|
pkgStatuses));
|
|
@@ -610,6 +665,22 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
AggregateConf instanceDefinition,
|
|
AggregateConf instanceDefinition,
|
|
MapOperations compOps)
|
|
MapOperations compOps)
|
|
throws SliderException, IOException {
|
|
throws SliderException, IOException {
|
|
|
|
+ // substitute CLUSTER_NAME into credentials
|
|
|
|
+ Map<String,List<String>> newcred = new HashMap<>();
|
|
|
|
+ for (Entry<String,List<String>> entry : instanceDefinition.getAppConf().credentials.entrySet()) {
|
|
|
|
+ List<String> resultList = new ArrayList<>();
|
|
|
|
+ for (String v : entry.getValue()) {
|
|
|
|
+ resultList.add(v.replaceAll(Pattern.quote("${CLUSTER_NAME}"),
|
|
|
|
+ clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
|
|
|
|
+ clusterName));
|
|
|
|
+ }
|
|
|
|
+ newcred.put(entry.getKey().replaceAll(Pattern.quote("${CLUSTER_NAME}"),
|
|
|
|
+ clusterName).replaceAll(Pattern.quote("${CLUSTER}"),
|
|
|
|
+ clusterName),
|
|
|
|
+ resultList);
|
|
|
|
+ }
|
|
|
|
+ instanceDefinition.getAppConf().credentials = newcred;
|
|
|
|
+
|
|
// generate and localize security stores
|
|
// generate and localize security stores
|
|
SecurityStore[] stores = generateSecurityStores(container, role,
|
|
SecurityStore[] stores = generateSecurityStores(container, role,
|
|
instanceDefinition, compOps);
|
|
instanceDefinition, compOps);
|
|
@@ -858,11 +929,12 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
.extractRole(container));
|
|
.extractRole(container));
|
|
if (role != null) {
|
|
if (role != null) {
|
|
String roleName = role.name;
|
|
String roleName = role.name;
|
|
- String label = getContainerLabel(container, roleName, role.group);
|
|
|
|
|
|
+ String roleGroup = role.group;
|
|
|
|
+ String label = getContainerLabel(container, roleName, roleGroup);
|
|
log.info("Rebuilding in-memory: container {} in role {} in cluster {}",
|
|
log.info("Rebuilding in-memory: container {} in role {} in cluster {}",
|
|
container.getId(), roleName, applicationId);
|
|
container.getId(), roleName, applicationId);
|
|
getComponentStatuses().put(label,
|
|
getComponentStatuses().put(label,
|
|
- new ComponentInstanceState(roleName, container.getId(),
|
|
|
|
|
|
+ new ComponentInstanceState(roleGroup, container.getId(),
|
|
applicationId));
|
|
applicationId));
|
|
} else {
|
|
} else {
|
|
log.warn("Role not found for container {} in cluster {}",
|
|
log.warn("Role not found for container {} in cluster {}",
|
|
@@ -983,7 +1055,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
|
|
|
|
StateAccessForProviders accessor = getAmState();
|
|
StateAccessForProviders accessor = getAmState();
|
|
CommandScript cmdScript = getScriptPathForMasterPackage(roleGroup);
|
|
CommandScript cmdScript = getScriptPathForMasterPackage(roleGroup);
|
|
- List<ComponentCommand> commands = getMetaInfo().getApplicationComponent(roleGroup).getCommands();
|
|
|
|
|
|
+ List<ComponentCommand> commands = getApplicationComponent(roleGroup).getCommands();
|
|
|
|
|
|
if (!isDockerContainer(roleGroup) && !isYarnDockerContainer(roleGroup)
|
|
if (!isDockerContainer(roleGroup) && !isYarnDockerContainer(roleGroup)
|
|
&& (cmdScript == null || cmdScript.getScript() == null)
|
|
&& (cmdScript == null || cmdScript.getScript() == null)
|
|
@@ -1261,7 +1333,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
}
|
|
}
|
|
|
|
|
|
private boolean isDockerContainer(String roleGroup) {
|
|
private boolean isDockerContainer(String roleGroup) {
|
|
- String type = getMetaInfo().getApplicationComponent(roleGroup).getType();
|
|
|
|
|
|
+ String type = getApplicationComponent(roleGroup).getType();
|
|
if (SliderUtils.isSet(type)) {
|
|
if (SliderUtils.isSet(type)) {
|
|
return type.toLowerCase().equals(SliderUtils.DOCKER) || type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
|
|
return type.toLowerCase().equals(SliderUtils.DOCKER) || type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
|
|
}
|
|
}
|
|
@@ -1269,7 +1341,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
}
|
|
}
|
|
|
|
|
|
private boolean isYarnDockerContainer(String roleGroup) {
|
|
private boolean isYarnDockerContainer(String roleGroup) {
|
|
- String type = getMetaInfo().getApplicationComponent(roleGroup).getType();
|
|
|
|
|
|
+ String type = getApplicationComponent(roleGroup).getType();
|
|
if (SliderUtils.isSet(type)) {
|
|
if (SliderUtils.isSet(type)) {
|
|
return type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
|
|
return type.toLowerCase().equals(SliderUtils.DOCKER_YARN);
|
|
}
|
|
}
|
|
@@ -1393,23 +1465,21 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
throw new IOException(e);
|
|
throw new IOException(e);
|
|
}
|
|
}
|
|
|
|
|
|
- for (ConfigFile configFile : getMetaInfo()
|
|
|
|
- .getComponentConfigFiles(client.getName())) {
|
|
|
|
|
|
+ for (ConfigFile configFile : getMetaInfo().getComponentConfigFiles(client.getName())) {
|
|
addNamedConfiguration(configFile.getDictionaryName(),
|
|
addNamedConfiguration(configFile.getDictionaryName(),
|
|
appConf.getGlobalOptions().options, configurations, tokens, null,
|
|
appConf.getGlobalOptions().options, configurations, tokens, null,
|
|
- client.getName());
|
|
|
|
|
|
+ client.getName(), client.getName());
|
|
if (appConf.getComponent(client.getName()) != null) {
|
|
if (appConf.getComponent(client.getName()) != null) {
|
|
addNamedConfiguration(configFile.getDictionaryName(),
|
|
addNamedConfiguration(configFile.getDictionaryName(),
|
|
appConf.getComponent(client.getName()).options, configurations,
|
|
appConf.getComponent(client.getName()).options, configurations,
|
|
- tokens, null, client.getName());
|
|
|
|
|
|
+ tokens, null, client.getName(), client.getName());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
//do a final replacement of re-used configs
|
|
//do a final replacement of re-used configs
|
|
dereferenceAllConfigs(configurations);
|
|
dereferenceAllConfigs(configurations);
|
|
|
|
|
|
- for (ConfigFile configFile : getMetaInfo()
|
|
|
|
- .getComponentConfigFiles(client.getName())) {
|
|
|
|
|
|
+ for (ConfigFile configFile : getMetaInfo().getComponentConfigFiles(client.getName())) {
|
|
ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
|
|
ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
|
|
|
|
|
|
Map<String, String> config = configurations.get(configFile.getDictionaryName());
|
|
Map<String, String> config = configurations.get(configFile.getDictionaryName());
|
|
@@ -1525,9 +1595,23 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
return workFolderExports;
|
|
return workFolderExports;
|
|
}
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
|
protected Metainfo getMetaInfo() {
|
|
protected Metainfo getMetaInfo() {
|
|
- return this.metaInfo;
|
|
|
|
|
|
+ return getMetaInfo(null);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ protected Metainfo getMetaInfo(String roleGroup) {
|
|
|
|
+ String mapKey = DEFAULT_METAINFO_MAP_KEY;
|
|
|
|
+ if (roleGroup != null) {
|
|
|
|
+ ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
|
|
|
|
+ mapKey = appConf.getComponentOpt(roleGroup, ROLE_PREFIX,
|
|
|
|
+ DEFAULT_METAINFO_MAP_KEY);
|
|
|
|
+ }
|
|
|
|
+ MetainfoHolder mh = this.metaInfoMap.get(mapKey);
|
|
|
|
+ if (mh == null) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ return mh.metaInfo;
|
|
}
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
@@ -1597,8 +1681,11 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
return defaultConfigMap;
|
|
return defaultConfigMap;
|
|
}
|
|
}
|
|
|
|
|
|
- protected Map<String, DefaultConfig> getDefaultConfigs() {
|
|
|
|
- return defaultConfigs;
|
|
|
|
|
|
+ protected Map<String, DefaultConfig> getDefaultConfigs(String roleGroup) {
|
|
|
|
+ ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
|
|
|
|
+ String mapKey = appConf.getComponentOpt(roleGroup, ROLE_PREFIX,
|
|
|
|
+ DEFAULT_METAINFO_MAP_KEY);
|
|
|
|
+ return metaInfoMap.get(mapKey).defaultConfigs;
|
|
}
|
|
}
|
|
|
|
|
|
private int getHeartbeatMonitorInterval() {
|
|
private int getHeartbeatMonitorInterval() {
|
|
@@ -1764,9 +1851,9 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
log.info("Status report: {}", status.toString());
|
|
log.info("Status report: {}", status.toString());
|
|
|
|
|
|
if (status.getConfigs() != null) {
|
|
if (status.getConfigs() != null) {
|
|
- Application application = getMetaInfo().getApplication();
|
|
|
|
|
|
+ Application application = getMetaInfo(componentGroup).getApplication();
|
|
|
|
|
|
- if ((!canAnyMasterPublishConfig() || canPublishConfig(componentGroup)) &&
|
|
|
|
|
|
+ if ((!canAnyMasterPublishConfig(componentGroup) || canPublishConfig(componentGroup)) &&
|
|
!getAmState().getAppConfSnapshot().getComponentOptBool(
|
|
!getAmState().getAppConfSnapshot().getComponentOptBool(
|
|
componentGroup, AgentKeys.AM_CONFIG_GENERATION, false)) {
|
|
componentGroup, AgentKeys.AM_CONFIG_GENERATION, false)) {
|
|
// If no Master can explicitly publish then publish if its a master
|
|
// If no Master can explicitly publish then publish if its a master
|
|
@@ -1914,7 +2001,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
String hostNamePattern = "${THIS_HOST}";
|
|
String hostNamePattern = "${THIS_HOST}";
|
|
Map<String, String> toPublish = new HashMap<String, String>();
|
|
Map<String, String> toPublish = new HashMap<String, String>();
|
|
|
|
|
|
- Application application = getMetaInfo().getApplication();
|
|
|
|
|
|
+ Application application = getMetaInfo(componentGroup).getApplication();
|
|
for (Component component : application.getComponents()) {
|
|
for (Component component : application.getComponents()) {
|
|
if (component.getName().equals(componentGroup)) {
|
|
if (component.getName().equals(componentGroup)) {
|
|
if (component.getComponentExports().size() > 0) {
|
|
if (component.getComponentExports().size() > 0) {
|
|
@@ -1965,8 +2052,8 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
String portVarFormat = "${site.%s}";
|
|
String portVarFormat = "${site.%s}";
|
|
String hostNamePattern = "${" + compGroup + "_HOST}";
|
|
String hostNamePattern = "${" + compGroup + "_HOST}";
|
|
|
|
|
|
- List<ExportGroup> appExportGroups = getMetaInfo().getApplication().getExportGroups();
|
|
|
|
- Component component = getMetaInfo().getApplicationComponent(compGroup);
|
|
|
|
|
|
+ List<ExportGroup> appExportGroups = getMetaInfo(compGroup).getApplication().getExportGroups();
|
|
|
|
+ Component component = getApplicationComponent(compGroup);
|
|
if (component != null && SliderUtils.isSet(component.getCompExports())
|
|
if (component != null && SliderUtils.isSet(component.getCompExports())
|
|
&& SliderUtils.isNotEmpty(appExportGroups)) {
|
|
&& SliderUtils.isNotEmpty(appExportGroups)) {
|
|
|
|
|
|
@@ -2068,7 +2155,11 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
* @return the component entry or null for no match
|
|
* @return the component entry or null for no match
|
|
*/
|
|
*/
|
|
protected Component getApplicationComponent(String roleGroup) {
|
|
protected Component getApplicationComponent(String roleGroup) {
|
|
- return getMetaInfo().getApplicationComponent(roleGroup);
|
|
|
|
|
|
+ Metainfo metainfo = getMetaInfo(roleGroup);
|
|
|
|
+ if (metainfo == null) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ return metainfo.getApplicationComponent(roleGroup);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -2137,9 +2228,9 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
*
|
|
*
|
|
* @return true if the condition holds
|
|
* @return true if the condition holds
|
|
*/
|
|
*/
|
|
- protected boolean canAnyMasterPublishConfig() {
|
|
|
|
|
|
+ protected boolean canAnyMasterPublishConfig(String roleGroup) {
|
|
if (canAnyMasterPublish == null) {
|
|
if (canAnyMasterPublish == null) {
|
|
- Application application = getMetaInfo().getApplication();
|
|
|
|
|
|
+ Application application = getMetaInfo(roleGroup).getApplication();
|
|
if (application == null) {
|
|
if (application == null) {
|
|
log.error("Malformed app definition: Expect application as root element in the metainfo.xml");
|
|
log.error("Malformed app definition: Expect application as root element in the metainfo.xml");
|
|
} else {
|
|
} else {
|
|
@@ -2214,7 +2305,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
cmd.setPkg(pkg);
|
|
cmd.setPkg(pkg);
|
|
Map<String, String> hostLevelParams = new TreeMap<String, String>();
|
|
Map<String, String> hostLevelParams = new TreeMap<String, String>();
|
|
hostLevelParams.put(JAVA_HOME, appConf.getGlobalOptions().getOption(JAVA_HOME, getJDKDir()));
|
|
hostLevelParams.put(JAVA_HOME, appConf.getGlobalOptions().getOption(JAVA_HOME, getJDKDir()));
|
|
- hostLevelParams.put(PACKAGE_LIST, getPackageList());
|
|
|
|
|
|
+ hostLevelParams.put(PACKAGE_LIST, getPackageList(roleGroup));
|
|
hostLevelParams.put(CONTAINER_ID, containerId);
|
|
hostLevelParams.put(CONTAINER_ID, containerId);
|
|
cmd.setHostLevelParams(hostLevelParams);
|
|
cmd.setHostLevelParams(hostLevelParams);
|
|
|
|
|
|
@@ -2263,7 +2354,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
cmd.setComponentName(roleName);
|
|
cmd.setComponentName(roleName);
|
|
cmd.setRole(roleName);
|
|
cmd.setRole(roleName);
|
|
Map<String, String> hostLevelParams = new TreeMap<String, String>();
|
|
Map<String, String> hostLevelParams = new TreeMap<String, String>();
|
|
- hostLevelParams.put(PACKAGE_LIST, getPackageList());
|
|
|
|
|
|
+ hostLevelParams.put(PACKAGE_LIST, getPackageList(roleGroup));
|
|
hostLevelParams.put(CONTAINER_ID, containerId);
|
|
hostLevelParams.put(CONTAINER_ID, containerId);
|
|
cmd.setHostLevelParams(hostLevelParams);
|
|
cmd.setHostLevelParams(hostLevelParams);
|
|
|
|
|
|
@@ -2283,7 +2374,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
configurations.get("global").put("exec_cmd", effectiveCommand.getExec());
|
|
configurations.get("global").put("exec_cmd", effectiveCommand.getExec());
|
|
|
|
|
|
cmd.setHostname(getClusterInfoPropertyValue(StatusKeys.INFO_AM_HOSTNAME));
|
|
cmd.setHostname(getClusterInfoPropertyValue(StatusKeys.INFO_AM_HOSTNAME));
|
|
- cmd.addContainerDetails(roleGroup, getMetaInfo());
|
|
|
|
|
|
+ cmd.addContainerDetails(roleGroup, getMetaInfo(roleGroup));
|
|
|
|
|
|
Map<String, String> dockerConfig = new HashMap<String, String>();
|
|
Map<String, String> dockerConfig = new HashMap<String, String>();
|
|
if(isYarnDockerContainer(roleGroup)){
|
|
if(isYarnDockerContainer(roleGroup)){
|
|
@@ -2366,8 +2457,8 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private String getPackageList() {
|
|
|
|
- return getPackageListFromApplication(getMetaInfo().getApplication());
|
|
|
|
|
|
+ private String getPackageList(String roleGroup) {
|
|
|
|
+ return getPackageListFromApplication(getMetaInfo(roleGroup).getApplication());
|
|
}
|
|
}
|
|
|
|
|
|
private void prepareExecutionCommand(ExecutionCommand cmd) {
|
|
private void prepareExecutionCommand(ExecutionCommand cmd) {
|
|
@@ -2532,7 +2623,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
private String getConfigFromMetaInfoWithAppConfigOverriding(String roleGroup,
|
|
private String getConfigFromMetaInfoWithAppConfigOverriding(String roleGroup,
|
|
String configName){
|
|
String configName){
|
|
ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
|
|
ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
|
|
- String containerName = getMetaInfo().getApplicationComponent(roleGroup)
|
|
|
|
|
|
+ String containerName = getApplicationComponent(roleGroup)
|
|
.getDockerContainers().get(0).getName();
|
|
.getDockerContainers().get(0).getName();
|
|
String composedConfigName = null;
|
|
String composedConfigName = null;
|
|
String appConfigValue = null;
|
|
String appConfigValue = null;
|
|
@@ -2673,7 +2764,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
|
|
|
|
cmd.setConfigurations(configurations);
|
|
cmd.setConfigurations(configurations);
|
|
// configurations.get("global").put("exec_cmd", startCommand.getExec());
|
|
// configurations.get("global").put("exec_cmd", startCommand.getExec());
|
|
- cmd.addContainerDetails(roleGroup, getMetaInfo());
|
|
|
|
|
|
+ cmd.addContainerDetails(roleGroup, getMetaInfo(roleGroup));
|
|
|
|
|
|
log.info("Docker- command: {}", cmd.toString());
|
|
log.info("Docker- command: {}", cmd.toString());
|
|
|
|
|
|
@@ -2683,7 +2774,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
private String getConfigFromMetaInfo(String roleGroup, String configName) {
|
|
private String getConfigFromMetaInfo(String roleGroup, String configName) {
|
|
String result = null;
|
|
String result = null;
|
|
|
|
|
|
- List<DockerContainer> containers = getMetaInfo().getApplicationComponent(
|
|
|
|
|
|
+ List<DockerContainer> containers = getApplicationComponent(
|
|
roleGroup).getDockerContainers();// to support multi container per
|
|
roleGroup).getDockerContainers();// to support multi container per
|
|
// component later
|
|
// component later
|
|
log.debug("Docker- containers metainfo: {}", containers.toString());
|
|
log.debug("Docker- containers metainfo: {}", containers.toString());
|
|
@@ -2985,10 +3076,11 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
|
|
|
|
for (String configType : configs) {
|
|
for (String configType : configs) {
|
|
addNamedConfiguration(configType, appConf.getGlobalOptions().options,
|
|
addNamedConfiguration(configType, appConf.getGlobalOptions().options,
|
|
- configurations, tokens, containerId, roleName);
|
|
|
|
|
|
+ configurations, tokens, containerId, roleName,
|
|
|
|
+ roleGroup);
|
|
if (appConf.getComponent(roleGroup) != null) {
|
|
if (appConf.getComponent(roleGroup) != null) {
|
|
addNamedConfiguration(configType, appConf.getComponent(roleGroup).options,
|
|
addNamedConfiguration(configType, appConf.getComponent(roleGroup).options,
|
|
- configurations, tokens, containerId, roleName);
|
|
|
|
|
|
+ configurations, tokens, containerId, roleName, roleGroup);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -3058,15 +3150,32 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
tokens.put("${NN_HOST}", URI.create(nnuri).getHost());
|
|
tokens.put("${NN_HOST}", URI.create(nnuri).getHost());
|
|
tokens.put("${ZK_HOST}", appConf.get(OptionKeys.ZOOKEEPER_HOSTS));
|
|
tokens.put("${ZK_HOST}", appConf.get(OptionKeys.ZOOKEEPER_HOSTS));
|
|
tokens.put("${DEFAULT_ZK_PATH}", appConf.get(OptionKeys.ZOOKEEPER_PATH));
|
|
tokens.put("${DEFAULT_ZK_PATH}", appConf.get(OptionKeys.ZOOKEEPER_PATH));
|
|
|
|
+ String prefix = appConf.getComponentOpt(componentGroup, ROLE_PREFIX,
|
|
|
|
+ null);
|
|
|
|
+ String dataDirSuffix = "";
|
|
|
|
+ if (prefix == null) {
|
|
|
|
+ prefix = "";
|
|
|
|
+ } else {
|
|
|
|
+ dataDirSuffix = "_" + SliderUtils.trimPrefix(prefix);
|
|
|
|
+ }
|
|
tokens.put("${DEFAULT_DATA_DIR}", getAmState()
|
|
tokens.put("${DEFAULT_DATA_DIR}", getAmState()
|
|
.getInternalsSnapshot()
|
|
.getInternalsSnapshot()
|
|
.getGlobalOptions()
|
|
.getGlobalOptions()
|
|
- .getMandatoryOption(InternalKeys.INTERNAL_DATA_DIR_PATH));
|
|
|
|
|
|
+ .getMandatoryOption(InternalKeys.INTERNAL_DATA_DIR_PATH) + dataDirSuffix);
|
|
tokens.put("${JAVA_HOME}", appConf.get(AgentKeys.JAVA_HOME));
|
|
tokens.put("${JAVA_HOME}", appConf.get(AgentKeys.JAVA_HOME));
|
|
tokens.put("${COMPONENT_NAME}", componentName);
|
|
tokens.put("${COMPONENT_NAME}", componentName);
|
|
|
|
+ tokens.put("${COMPONENT_NAME.lc}", componentName.toLowerCase());
|
|
|
|
+ tokens.put("${COMPONENT_PREFIX}", prefix);
|
|
|
|
+ tokens.put("${COMPONENT_PREFIX.lc}", prefix.toLowerCase());
|
|
if (!componentName.equals(componentGroup) && componentName.startsWith(componentGroup)) {
|
|
if (!componentName.equals(componentGroup) && componentName.startsWith(componentGroup)) {
|
|
tokens.put("${COMPONENT_ID}", componentName.substring(componentGroup.length()));
|
|
tokens.put("${COMPONENT_ID}", componentName.substring(componentGroup.length()));
|
|
}
|
|
}
|
|
|
|
+ tokens.put("${CLUSTER_NAME}", getClusterName());
|
|
|
|
+ tokens.put("${CLUSTER_NAME.lc}", getClusterName().toLowerCase());
|
|
|
|
+ tokens.put("${APP_NAME}", getClusterName());
|
|
|
|
+ tokens.put("${APP_NAME.lc}", getClusterName().toLowerCase());
|
|
|
|
+ tokens.put("${APP_COMPONENT_NAME}", componentName);
|
|
|
|
+ tokens.put("${APP_COMPONENT_NAME.lc}", componentName.toLowerCase());
|
|
return tokens;
|
|
return tokens;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -3091,12 +3200,12 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
List<String> configList = new ArrayList<String>();
|
|
List<String> configList = new ArrayList<String>();
|
|
configList.add(GLOBAL_CONFIG_TAG);
|
|
configList.add(GLOBAL_CONFIG_TAG);
|
|
|
|
|
|
- List<ConfigFile> configFiles = getMetaInfo().getApplication().getConfigFiles();
|
|
|
|
|
|
+ List<ConfigFile> configFiles = getMetaInfo(roleGroup).getApplication().getConfigFiles();
|
|
for (ConfigFile configFile : configFiles) {
|
|
for (ConfigFile configFile : configFiles) {
|
|
log.info("Expecting config type {}.", configFile.getDictionaryName());
|
|
log.info("Expecting config type {}.", configFile.getDictionaryName());
|
|
configList.add(configFile.getDictionaryName());
|
|
configList.add(configFile.getDictionaryName());
|
|
}
|
|
}
|
|
- for (Component component : getMetaInfo().getApplication().getComponents()) {
|
|
|
|
|
|
+ for (Component component : getMetaInfo(roleGroup).getApplication().getComponents()) {
|
|
if (!component.getName().equals(roleGroup)) {
|
|
if (!component.getName().equals(roleGroup)) {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
@@ -3121,7 +3230,7 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
private void addNamedConfiguration(String configName, Map<String, String> sourceConfig,
|
|
private void addNamedConfiguration(String configName, Map<String, String> sourceConfig,
|
|
Map<String, Map<String, String>> configurations,
|
|
Map<String, Map<String, String>> configurations,
|
|
Map<String, String> tokens, String containerId,
|
|
Map<String, String> tokens, String containerId,
|
|
- String roleName) {
|
|
|
|
|
|
+ String roleName, String roleGroup) {
|
|
Map<String, String> config = new HashMap<String, String>();
|
|
Map<String, String> config = new HashMap<String, String>();
|
|
if (configName.equals(GLOBAL_CONFIG_TAG)) {
|
|
if (configName.equals(GLOBAL_CONFIG_TAG)) {
|
|
addDefaultGlobalConfig(config, containerId, roleName);
|
|
addDefaultGlobalConfig(config, containerId, roleName);
|
|
@@ -3150,9 +3259,9 @@ public class AgentProviderService extends AbstractProviderService implements
|
|
}
|
|
}
|
|
|
|
|
|
//apply defaults only if the key is not present and value is not empty
|
|
//apply defaults only if the key is not present and value is not empty
|
|
- if (getDefaultConfigs().containsKey(configName)) {
|
|
|
|
|
|
+ if (getDefaultConfigs(roleGroup).containsKey(configName)) {
|
|
log.info("Adding default configs for type {}.", configName);
|
|
log.info("Adding default configs for type {}.", configName);
|
|
- for (PropertyInfo defaultConfigProp : getDefaultConfigs().get(configName).getPropertyInfos()) {
|
|
|
|
|
|
+ for (PropertyInfo defaultConfigProp : getDefaultConfigs(roleGroup).get(configName).getPropertyInfos()) {
|
|
if (!config.containsKey(defaultConfigProp.getName())) {
|
|
if (!config.containsKey(defaultConfigProp.getName())) {
|
|
if (!defaultConfigProp.getName().isEmpty() &&
|
|
if (!defaultConfigProp.getName().isEmpty() &&
|
|
defaultConfigProp.getValue() != null &&
|
|
defaultConfigProp.getValue() != null &&
|