You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sm...@apache.org on 2016/12/08 00:23:58 UTC

ambari git commit: AMBARI-19077: Ambari-server: Gather dependent configuration types and password properties for a service component

Repository: ambari
Updated Branches:
  refs/heads/trunk 0fa8238c7 -> 9652b2313


AMBARI-19077: Ambari-server: Gather dependent configuration types and password properties for a service component


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/9652b231
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/9652b231
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/9652b231

Branch: refs/heads/trunk
Commit: 9652b2313e52f724dbffd07bab54901c3b9b060c
Parents: 0fa8238
Author: Nahappan Somasundaram <ns...@hortonworks.com>
Authored: Sat Dec 3 10:37:53 2016 -0800
Committer: Nahappan Somasundaram <ns...@hortonworks.com>
Committed: Wed Dec 7 16:23:46 2016 -0800

----------------------------------------------------------------------
 .../ambari_agent/CustomServiceOrchestrator.py   | 156 +++++++++++++------
 .../ambari/server/agent/ExecutionCommand.java   |  33 ++++
 .../AmbariManagementControllerImpl.java         |  13 ++
 .../ambari/server/state/ConfigHelper.java       |  45 ++++++
 .../ambari/server/state/PropertyInfo.java       |  15 ++
 5 files changed, 218 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/9652b231/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py b/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
index f9ed4cf..5fd3068 100644
--- a/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
+++ b/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
@@ -72,6 +72,9 @@ class CustomServiceOrchestrator():
   DEFAULT_CREDENTIAL_CONF_DIR = '/var/lib/ambari-agent/cred/conf'
   DEFAULT_CREDENTIAL_SHELL_CMD = 'org.apache.hadoop.security.alias.CredentialShell'
 
+  # The property name used by the hadoop credential provider
+  CREDENTIAL_PROVIDER_PROPERTY_NAME = 'hadoop.security.credential.provider.path'
+
   def __init__(self, config, controller):
     self.config = config
     self.tmp_dir = config.get('agent', 'prefix')
@@ -147,21 +150,97 @@ class CustomServiceOrchestrator():
 
     return conf_dir
 
-  def getAffectedConfigTypes(self, commandJson):
+  def getConfigTypeCredentials(self, commandJson):
     """
     Gets the affected config types for the service in this command
+    with the password aliases and values.
+
+    Input:
+    {
+        "config-type1" : {
+          "password_key_name1":"password_value_name1",
+          "password_key_name2":"password_value_name2",
+            :
+        },
+        "config-type2" : {
+          "password_key_name1":"password_value_name1",
+          "password_key_name2":"password_value_name2",
+            :
+        },
+           :
+    }
+
+    Output:
+    {
+        "config-type1" : {
+          "alias1":"password1",
+          "alias2":"password2",
+            :
+        },
+        "config-type2" : {
+          "alias1":"password1",
+          "alias2":"password2",
+            :
+        },
+           :
+    }
+
+    If password_key_name is the same as password_value_name, then password_key_name is the password alias itself.
+    The value it points to is the password value.
+
+    If password_key_name is not the same as the password_value_name, then password_key_name points to the alias.
+    The value is pointed to by password_value_name.
+
+    For example:
+    Input:
+    {
+      "oozie-site" : {"oozie.service.JPAService.jdbc.password" : "oozie.service.JPAService.jdbc.password"},
+      "admin-properties" {"db_user":"db_password", "ranger.jpa.jdbc.credential.alias:ranger-admin-site" : "db_password"}
+    }
+
+    Output:
+    {
+      "oozie-site" : {"oozie.service.JPAService.jdbc.password" : "MyOozieJdbcPassword"},
+      "admin-properties" {"rangerdba" : "MyRangerDbaPassword", "rangeradmin":"MyRangerDbaPassword"},
+    }
 
     :param commandJson:
     :return:
     """
-    return commandJson.get('configuration_attributes')
-
-  def getCredentialProviderPropertyName(self):
-    """
-    Gets the property name used by the hadoop credential provider
-    :return:
-    """
-    return 'hadoop.security.credential.provider.path'
+    configtype_credentials = {}
+    if 'configuration_credentials' in commandJson:
+      for config_type, password_properties in commandJson['configuration_credentials'].items():
+        if config_type in commandJson['configurations']:
+          value_names = []
+          config = commandJson['configurations'][config_type]
+          credentials = {}
+          for key_name, value_name in password_properties.items():
+            if key_name == value_name:
+              if value_name in config:
+                # password name is the alias
+                credentials[key_name] = config[value_name]
+                value_names.append(value_name) # Gather the value_name for deletion
+            else:
+              keyname_keyconfig = key_name.split(':')
+              key_name = keyname_keyconfig[0]
+              # if the key is in another configuration (cross reference),
+              # get the value of the key from that configuration
+              if (len(keyname_keyconfig) > 1):
+                if keyname_keyconfig[1] not in commandJson['configurations']:
+                  continue
+                key_config = commandJson['configurations'][keyname_keyconfig[1]]
+              else:
+                key_config = config
+              if key_name in key_config and value_name in config:
+                # password name points to the alias
+                credentials[key_config[key_name]] = config[value_name]
+                value_names.append(value_name) # Gather the value_name for deletion
+          if len(credentials) > 0:
+            configtype_credentials[config_type] = credentials
+          for value_name in value_names:
+            # Remove the clear text password
+            config.pop(value_name, None)
+    return configtype_credentials
 
   def generateJceks(self, commandJson):
     """
@@ -178,16 +257,6 @@ class CustomServiceOrchestrator():
 
     logger.info('generateJceks: roleCommand={0}'.format(roleCommand))
 
-    # Password properties for a config type, if present,
-    # are under configuration_attributes:config_type:hidden:{prop1:attributes1, prop2, attributes2}
-    passwordProperties = {}
-    config_types = self.getAffectedConfigTypes(commandJson)
-    for config_type in config_types:
-      elem = config_types.get(config_type)
-      hidden = elem.get('hidden')
-      if hidden is not None:
-        passwordProperties[config_type] = hidden
-
     # Set up the variables for the external command to generate a JCEKS file
     java_home = commandJson['hostLevelParams']['java_home']
     java_bin = '{java_home}/bin/java'.format(java_home=java_home)
@@ -196,31 +265,30 @@ class CustomServiceOrchestrator():
     serviceName = commandJson['serviceName']
 
     # Gather the password values and remove them from the configuration
-    configs = commandJson.get('configurations')
-    for key, value in passwordProperties.items():
-      config = configs.get(key)
-      if config is not None:
-        file_path = os.path.join(self.getProviderDirectory(serviceName), "{0}.jceks".format(key))
-        if os.path.exists(file_path):
-          os.remove(file_path)
-        provider_path = 'jceks://file{file_path}'.format(file_path=file_path)
-        logger.info('provider_path={0}'.format(provider_path))
-        for alias in value:
-          pwd = config.get(alias)
-          if pwd is not None:
-            # Remove the clear text password
-            config.pop(alias, None)
-            # Add JCEKS provider path instead
-            config[self.getCredentialProviderPropertyName()] = provider_path
-            logger.debug("config={0}".format(config))
-            protected_pwd = PasswordString(pwd)
-            # Generate the JCEKS file
-            cmd = (java_bin, '-cp', cs_lib_path, self.credential_shell_cmd, 'create',
-                   alias, '-value', protected_pwd, '-provider', provider_path)
-            logger.info(cmd)
-            cmd_result = subprocess.call(cmd)
-            logger.info('cmd_result = {0}'.format(cmd_result))
-            os.chmod(file_path, 0644) # group and others should have read access so that the service user can read
+    provider_paths = [] # A service may depend on multiple configs
+    configtype_credentials = self.getConfigTypeCredentials(commandJson)
+    for config_type, credentials in configtype_credentials.items():
+      config = commandJson['configurations'][config_type]
+      file_path = os.path.join(self.getProviderDirectory(serviceName), "{0}.jceks".format(config_type))
+      if os.path.exists(file_path):
+        os.remove(file_path)
+      provider_path = 'jceks://file{file_path}'.format(file_path=file_path)
+      provider_paths.append(provider_path)
+      logger.info('provider_path={0}'.format(provider_path))
+      for alias, pwd in credentials.items():
+        logger.debug("config={0}".format(config))
+        protected_pwd = PasswordString(pwd)
+        # Generate the JCEKS file
+        cmd = (java_bin, '-cp', cs_lib_path, self.credential_shell_cmd, 'create',
+               alias, '-value', protected_pwd, '-provider', provider_path)
+        logger.info(cmd)
+        cmd_result = subprocess.call(cmd)
+        logger.info('cmd_result = {0}'.format(cmd_result))
+        os.chmod(file_path, 0644) # group and others should have read access so that the service user can read
+
+    if provider_paths:
+      # Add JCEKS provider paths instead
+      config[self.CREDENTIAL_PROVIDER_PROPERTY_NAME] = ','.join(provider_paths)
 
     return cmd_result
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/9652b231/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java b/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
index e46167a..5c4f08e 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java
@@ -119,6 +119,39 @@ public class ExecutionCommand extends AgentCommand {
   @SerializedName("credentialStoreEnabled")
   private String credentialStoreEnabled;
 
+  /**
+   * Map of config type to list of password properties
+   *   <pre>
+   *     {@code
+   *       {
+   *         "config_type1" :
+   *           {
+   *             "password_alias_name1:type1":"password_value_name1",
+   *             "password_alias_name2:type2":"password_value_name2",
+   *                 :
+   *           },
+   *         "config_type2" :
+   *           {
+   *             "password_alias_name1:type1":"password_value_name1",
+   *             "password_alias_name2:type2":"password_value_name2",
+   *                 :
+   *           },
+   *                 :
+   *       }
+   *     }
+   *   </pre>
+   */
+  @SerializedName("configuration_credentials")
+  private Map<String, Map<String, String>> configurationCredentials;
+
+  public void setConfigurationCredentials(Map<String, Map<String, String>> configurationCredentials) {
+    this.configurationCredentials = configurationCredentials;
+  }
+
+  public Map<String, Map<String, String>> getConfigurationCredentials() {
+    return this.configurationCredentials;
+  }
+
   public String getCommandId() {
     return commandId;
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/9652b231/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
index 63b6932..1777689 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariManagementControllerImpl.java
@@ -341,6 +341,8 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
   @Inject
   private AmbariActionExecutionHelper actionExecutionHelper;
 
+  private Map<String, Map<String, Map<String, String>>> configCredentialsForService = new HashMap<>();
+
   @Inject
   public AmbariManagementControllerImpl(ActionManager actionManager,
       Clusters clusters, Injector injector) throws Exception {
@@ -2115,6 +2117,17 @@ public class AmbariManagementControllerImpl implements AmbariManagementControlle
     Service clusterService = cluster.getService(serviceName);
     execCmd.setCredentialStoreEnabled(String.valueOf(clusterService.isCredentialStoreEnabled()));
 
+    // Get the map of service config type to password properties for the service
+    Map<String, Map<String, String>> configCredentials;
+    configCredentials = configCredentialsForService.get(clusterService.getName());
+    if (configCredentials == null) {
+      configCredentials = configHelper.getPropertiesWithPropertyType(stackId, clusterService,
+              PropertyType.PASSWORD);
+      configCredentialsForService.put(clusterService.getName(), configCredentials);
+    }
+
+    execCmd.setConfigurationCredentials(configCredentials);
+
     // Create a local copy for each command
     Map<String, String> commandParams = new TreeMap<String, String>();
     if (commandParamsInp != null) { // if not defined

http://git-wip-us.apache.org/repos/asf/ambari/blob/9652b231/ambari-server/src/main/java/org/apache/ambari/server/state/ConfigHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/ConfigHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/state/ConfigHelper.java
index f975053..8022457 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/ConfigHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/ConfigHelper.java
@@ -41,6 +41,7 @@ import org.apache.ambari.server.orm.entities.ClusterConfigEntity;
 import org.apache.ambari.server.state.PropertyInfo.PropertyType;
 import org.apache.ambari.server.state.configgroup.ConfigGroup;
 import org.apache.ambari.server.utils.SecretReference;
+import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -488,6 +489,50 @@ public class ConfigHelper {
     return result;
   }
 
+  /**
+   * Gets a map of config types to password property names to password property value names.
+   *
+   * @param stackId
+   * @param service
+   * @param propertyType
+   * @return
+   * @throws AmbariException
+     */
+  public Map<String, Map<String, String>> getPropertiesWithPropertyType(StackId stackId, Service service, PropertyType propertyType)
+          throws AmbariException {
+    StackInfo stack = ambariMetaInfo.getStack(stackId.getStackName(), stackId.getStackVersion());
+    Map<String, Map<String, String>> result = new HashMap<>();
+    Map<String, String> passwordProperties;
+    Set<PropertyInfo> serviceProperties = ambariMetaInfo.getServiceProperties(stack.getName(), stack.getVersion(), service.getName());
+    for (PropertyInfo serviceProperty : serviceProperties) {
+      if (serviceProperty.getPropertyTypes().contains(propertyType)) {
+        String stackPropertyConfigType = fileNameToConfigType(serviceProperty.getFilename());
+        passwordProperties = result.get(stackPropertyConfigType);
+        if (passwordProperties == null) {
+          passwordProperties = new HashMap<>();
+          result.put(stackPropertyConfigType, passwordProperties);
+        }
+        // If the password property is used by another property, it means the password property
+        // is a password value name while the use is the password alias name. If the user property
+        // is from another config type, include that in the password alias name as name:type.
+        if (serviceProperty.getUsedByProperties().size() > 0) {
+          for (PropertyDependencyInfo usedByProperty : serviceProperty.getUsedByProperties()) {
+            String propertyName = usedByProperty.getName();
+            if (!StringUtils.isEmpty(usedByProperty.getType())) {
+              propertyName += ':' + usedByProperty.getType();
+            }
+            passwordProperties.put(propertyName, serviceProperty.getName());
+          }
+        }
+        else {
+          passwordProperties.put(serviceProperty.getName(), serviceProperty.getName());
+        }
+      }
+    }
+
+    return result;
+  }
+
   public Set<String> getPropertyValuesWithPropertyType(StackId stackId, PropertyType propertyType,
       Cluster cluster, Map<String, DesiredConfig> desiredConfigs) throws AmbariException {
     StackInfo stack = ambariMetaInfo.getStack(stackId.getStackName(), stackId.getStackVersion());

http://git-wip-us.apache.org/repos/asf/ambari/blob/9652b231/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
index 81de76c..01c8a89 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/PropertyInfo.java
@@ -79,6 +79,17 @@ public class PropertyInfo {
   private Set<PropertyDependencyInfo> dependedByProperties =
     new HashSet<PropertyDependencyInfo>();
 
+  /**
+   * The list of properties that use this property.
+   * Password properties may be used by other properties in
+   * the same config type or different config type, typically
+   * when asking for user name and password pairs.
+   */
+  @XmlElementWrapper(name="used-by")
+  @XmlElement(name = "property")
+  private Set<PropertyDependencyInfo> usedByProperties =
+          new HashSet<>();
+
   //This method is called after all the properties (except IDREF) are unmarshalled for this object,
   //but before this object is set to the parent object.
   void afterUnmarshal(Unmarshaller unmarshaller, Object parent) {
@@ -99,6 +110,10 @@ public class PropertyInfo {
     this.name = name;
   }
 
+  public Set<PropertyDependencyInfo> getUsedByProperties() {
+    return usedByProperties;
+  }
+
   public String getValue() {
     return value;
   }