|
@@ -46,27 +46,24 @@ import org.apache.slider.providers.ProviderCore;
|
|
|
import org.apache.slider.providers.ProviderRole;
|
|
|
import org.apache.slider.providers.ProviderUtils;
|
|
|
import org.apache.slider.server.appmaster.state.RoleInstance;
|
|
|
-import org.apache.slider.server.appmaster.state.StateAccessForProviders;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URL;
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Date;
|
|
|
-import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
-import java.util.Locale;
|
|
|
import java.util.Map;
|
|
|
import java.util.Map.Entry;
|
|
|
import java.util.Scanner;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.regex.Pattern;
|
|
|
|
|
|
-import static org.apache.slider.api.RoleKeys.ROLE_PREFIX;
|
|
|
-
|
|
|
public class DockerProviderService extends AbstractProviderService implements
|
|
|
ProviderCore,
|
|
|
DockerKeys,
|
|
@@ -77,10 +74,16 @@ public class DockerProviderService extends AbstractProviderService implements
|
|
|
private static final ProviderUtils providerUtils = new ProviderUtils(log);
|
|
|
private static final String EXPORT_GROUP = "quicklinks";
|
|
|
private static final String APPLICATION_TAG = "application";
|
|
|
+ private static final String HOST_KEY_FORMAT = "${%s_HOST}";
|
|
|
+ private static final String IP_KEY_FORMAT = "${%s_IP}";
|
|
|
+ private static final String VARIABLE_INDICATOR = "${";
|
|
|
|
|
|
private String clusterName = null;
|
|
|
private SliderFileSystem fileSystem = null;
|
|
|
|
|
|
+ private final Map<String, Set<ExportEntry>> exportMap =
|
|
|
+ new ConcurrentHashMap<>();
|
|
|
+
|
|
|
protected DockerProviderService() {
|
|
|
super("DockerProviderService");
|
|
|
}
|
|
@@ -118,9 +121,6 @@ public class DockerProviderService extends AbstractProviderService implements
|
|
|
String roleName = providerRole.name;
|
|
|
String roleGroup = providerRole.group;
|
|
|
|
|
|
- initializeApplicationConfiguration(instanceDefinition, fileSystem,
|
|
|
- roleGroup);
|
|
|
-
|
|
|
log.info("Build launch context for Docker");
|
|
|
log.debug(instanceDefinition.toString());
|
|
|
|
|
@@ -279,6 +279,23 @@ public class DockerProviderService extends AbstractProviderService implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void notifyContainerCompleted(ContainerId containerId) {
|
|
|
+ if (containerId != null) {
|
|
|
+ String containerIdStr = containerId.toString();
|
|
|
+ log.info("Removing container exports for {}", containerIdStr);
|
|
|
+ for (Set<ExportEntry> exportEntries : exportMap.values()) {
|
|
|
+ for (Iterator<ExportEntry> iter = exportEntries.iterator();
|
|
|
+ iter.hasNext();) {
|
|
|
+ ExportEntry entry = iter.next();
|
|
|
+ if (containerIdStr.equals(entry.getContainerId())) {
|
|
|
+ iter.remove();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public boolean processContainerStatus(ContainerId containerId,
|
|
|
ContainerStatus status) {
|
|
@@ -301,7 +318,7 @@ public class DockerProviderService extends AbstractProviderService implements
|
|
|
containerIdStr, roleName, status.getIPs(), status.getHost());
|
|
|
|
|
|
publishExportGroups(containerIdStr, roleName, roleGroup,
|
|
|
- status.getHost());
|
|
|
+ status.getHost(), status.getIPs());
|
|
|
return false;
|
|
|
}
|
|
|
|
|
@@ -312,17 +329,13 @@ public class DockerProviderService extends AbstractProviderService implements
|
|
|
* are substituted with the actual hostnames of the containers.
|
|
|
*/
|
|
|
protected void publishExportGroups(String containerId,
|
|
|
- String roleName, String roleGroup, String thisHost) {
|
|
|
+ String roleName, String roleGroup, String thisHost, List<String> ips) {
|
|
|
ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
|
|
|
ConfTreeOperations internalsConf = getAmState().getInternalsSnapshot();
|
|
|
|
|
|
Map<String, String> exports = providerUtils.getExports(
|
|
|
getAmState().getAppConfSnapshot(), roleGroup);
|
|
|
|
|
|
- String hostKeyFormat = "${%s_HOST}";
|
|
|
- String hostNameKeyFormat = "${%s_HOSTNAME}";
|
|
|
- String ipKeyFormat = "${%s_IP}";
|
|
|
-
|
|
|
// publish export groups if any
|
|
|
Map<String, String> standardTokens = providerUtils.getStandardTokenMap(
|
|
|
appConf, internalsConf, roleName, roleGroup, containerId,
|
|
@@ -331,38 +344,33 @@ public class DockerProviderService extends AbstractProviderService implements
|
|
|
appConf.getComponent(roleGroup).options, standardTokens);
|
|
|
replaceTokens.putAll(standardTokens);
|
|
|
|
|
|
- String rolePrefix = appConf.getComponentOpt(roleGroup, ROLE_PREFIX, "");
|
|
|
- for (Map.Entry<String, Map<String, ClusterNode>> entry :
|
|
|
- getAmState().getRoleClusterNodeMapping().entrySet()) {
|
|
|
- String otherRolePrefix = appConf.getComponentOpt(entry.getKey(),
|
|
|
- ROLE_PREFIX, "");
|
|
|
- if (!otherRolePrefix.equals(rolePrefix)) {
|
|
|
- // hostname replacements are only made within role prefix groups
|
|
|
- continue;
|
|
|
- }
|
|
|
- String key = entry.getKey();
|
|
|
- if (!rolePrefix.isEmpty()) {
|
|
|
- if (!key.startsWith(rolePrefix)) {
|
|
|
- log.warn("Something went wrong, {} doesn't start with {}", key,
|
|
|
- rolePrefix);
|
|
|
- continue;
|
|
|
- }
|
|
|
- key = key.substring(rolePrefix.length());
|
|
|
+ String roleNameKey = providerUtils.getNameKey(roleName, roleGroup,
|
|
|
+ appConf);
|
|
|
+ String roleNameIPKey = null;
|
|
|
+ if (roleNameKey != null) {
|
|
|
+ replaceTokens.put(String.format(HOST_KEY_FORMAT, roleNameKey), thisHost);
|
|
|
+ roleNameIPKey = Pattern.quote(String.format(IP_KEY_FORMAT, roleNameKey));
|
|
|
+ } else {
|
|
|
+ // should not happen, but log if it does
|
|
|
+ log.info("Not replacing HOST or IP tokens because key was null for {}",
|
|
|
+ roleName);
|
|
|
+ }
|
|
|
+ String roleGroupKey = providerUtils.getGroupKey(roleGroup, appConf);
|
|
|
+ String roleGroupIPKey = null;
|
|
|
+ if (roleGroupKey != null) {
|
|
|
+ if (roleNameKey == null || !roleGroupKey.equals(roleNameKey)) {
|
|
|
+ replaceTokens.put(String.format(HOST_KEY_FORMAT, roleGroupKey),
|
|
|
+ thisHost);
|
|
|
+ roleGroupIPKey = Pattern.quote(String.format(IP_KEY_FORMAT,
|
|
|
+ roleGroupKey));
|
|
|
}
|
|
|
- key = key.toUpperCase(Locale.ENGLISH);
|
|
|
- String host = providerUtils.getHostsList(
|
|
|
- entry.getValue().values(), true).iterator().next();
|
|
|
- replaceTokens.put(String.format(hostKeyFormat, key), host);
|
|
|
- String hostName = providerUtils.getHostNamesList(
|
|
|
- entry.getValue().values()).iterator().next();
|
|
|
- replaceTokens.put(String.format(hostNameKeyFormat, key), hostName);
|
|
|
- String ip = providerUtils.getIPsList(
|
|
|
- entry.getValue().values()).iterator().next();
|
|
|
- replaceTokens.put(String.format(ipKeyFormat, key), ip);
|
|
|
+ } else {
|
|
|
+ // should not happen, but log if it does
|
|
|
+ log.info("Not replacing HOST or IP tokens because key was null for {}",
|
|
|
+ roleGroup);
|
|
|
}
|
|
|
replaceTokens.put("${THIS_HOST}", thisHost);
|
|
|
|
|
|
- Map<String, List<ExportEntry>> entries = new HashMap<>();
|
|
|
for (Entry<String, String> export : exports.entrySet()) {
|
|
|
String value = export.getValue();
|
|
|
// replace host names and site properties
|
|
@@ -372,18 +380,53 @@ public class DockerProviderService extends AbstractProviderService implements
|
|
|
value = value.replaceAll(Pattern.quote(token), entry.getValue());
|
|
|
}
|
|
|
}
|
|
|
- ExportEntry entry = new ExportEntry();
|
|
|
- entry.setLevel(APPLICATION_TAG);
|
|
|
- entry.setValue(value);
|
|
|
- entry.setUpdatedTime(new Date().toString());
|
|
|
- // over-write, app exports are singletons
|
|
|
- entries.put(export.getKey(), new ArrayList(Arrays.asList(entry)));
|
|
|
- log.info("Preparing to publish. Key {} and Value {}",
|
|
|
- export.getKey(), value);
|
|
|
+ Set<String> values = new HashSet<>();
|
|
|
+ for (String ip : ips) {
|
|
|
+ values.add(substituteIP(roleNameIPKey, roleGroupIPKey, ip, value));
|
|
|
+ }
|
|
|
+ for (String exportValue : values) {
|
|
|
+ if (exportValue.contains(VARIABLE_INDICATOR)) {
|
|
|
+ // not all variables have been substituted, so do not export
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ ExportEntry entry = new ExportEntry();
|
|
|
+ entry.setContainerId(containerId);
|
|
|
+ entry.setLevel(APPLICATION_TAG);
|
|
|
+ entry.setValue(exportValue);
|
|
|
+ entry.setUpdatedTime(new Date().toString());
|
|
|
+ Set<ExportEntry> exportEntries = getExportEntries(export.getKey());
|
|
|
+ exportEntries.add(entry);
|
|
|
+ log.info("Preparing to publish for {}. Key {} and Value {}",
|
|
|
+ roleName, export.getKey(), entry);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (!exportMap.isEmpty()) {
|
|
|
+ providerUtils.publishExportGroup(exportMap, getAmState(), EXPORT_GROUP);
|
|
|
}
|
|
|
- if (!entries.isEmpty()) {
|
|
|
- providerUtils.publishExportGroup(entries, getAmState(), EXPORT_GROUP);
|
|
|
+ }
|
|
|
+
|
|
|
+ protected String substituteIP(String roleNameIPKey, String roleGroupIPKey,
|
|
|
+ String ip, String value) {
|
|
|
+ if (roleNameIPKey != null) {
|
|
|
+ value = value.replaceAll(roleNameIPKey, ip);
|
|
|
}
|
|
|
+ if (roleGroupIPKey != null) {
|
|
|
+ value = value.replaceAll(roleGroupIPKey, ip);
|
|
|
+ }
|
|
|
+ return value;
|
|
|
+ }
|
|
|
+
|
|
|
+ protected Set<ExportEntry> getExportEntries(String key) {
|
|
|
+ if (!this.exportMap.containsKey(key)) {
|
|
|
+ synchronized (this.exportMap) {
|
|
|
+ if (!this.exportMap.containsKey(key)) {
|
|
|
+ this.exportMap.put(key, Collections.newSetFromMap(
|
|
|
+ new ConcurrentHashMap<>()));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ return this.exportMap.get(key);
|
|
|
}
|
|
|
|
|
|
@Override
|