You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by mo...@apache.org on 2017/10/05 14:19:20 UTC

[2/2] knox git commit: KNOX-1064 - Externalize Hadoop Service Configuration Details and Service URL Creation (Phil Zampino via Sandeep More)

KNOX-1064 - Externalize Hadoop Service Configuration Details and Service URL Creation (Phil Zampino via Sandeep More)


Project: http://git-wip-us.apache.org/repos/asf/knox/repo
Commit: http://git-wip-us.apache.org/repos/asf/knox/commit/7b401def
Tree: http://git-wip-us.apache.org/repos/asf/knox/tree/7b401def
Diff: http://git-wip-us.apache.org/repos/asf/knox/diff/7b401def

Branch: refs/heads/master
Commit: 7b401def625630cbf1f9ee5f8993bbcb3269c222
Parents: d762ed3
Author: Sandeep More <mo...@apache.org>
Authored: Thu Oct 5 10:19:07 2017 -0400
Committer: Sandeep More <mo...@apache.org>
Committed: Thu Oct 5 10:19:07 2017 -0400

----------------------------------------------------------------------
 .../discovery/ambari/AmbariCluster.java         |   7 +-
 .../discovery/ambari/AmbariComponent.java       |  27 +-
 .../ambari/AmbariDynamicServiceURLCreator.java  | 151 ++++
 .../ambari/AmbariServiceDiscovery.java          |  58 +-
 .../ambari/AmbariServiceDiscoveryMessages.java  |  64 +-
 .../ambari/AmbariServiceURLCreator.java         | 184 ----
 .../ambari/ConditionalValueHandler.java         |  24 +
 .../discovery/ambari/PropertyEqualsHandler.java |  76 ++
 .../ambari/ServiceURLPropertyConfig.java        | 324 +++++++
 .../discovery/ambari/SimpleValueHandler.java    |  32 +
 ...iscovery-component-config-mapping.properties |  36 +
 .../ambari-service-discovery-url-mappings.xml   | 398 +++++++++
 .../AmbariDynamicServiceURLCreatorTest.java     | 876 +++++++++++++++++++
 .../ambari/AmbariServiceDiscoveryTest.java      |   4 +-
 .../simple/SimpleDescriptorHandler.java         |  68 +-
 .../simple/SimpleDescriptorMessages.java        |   8 +-
 .../simple/SimpleDescriptorHandlerTest.java     | 327 +++++--
 17 files changed, 2335 insertions(+), 329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariCluster.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariCluster.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariCluster.java
index 6eaabd3..eb84433 100644
--- a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariCluster.java
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariCluster.java
@@ -27,7 +27,7 @@ class AmbariCluster implements ServiceDiscovery.Cluster {
 
     private String name = null;
 
-    private AmbariServiceURLCreator urlCreator = new AmbariServiceURLCreator();
+    private AmbariDynamicServiceURLCreator urlCreator;
 
     private Map<String, Map<String, ServiceConfiguration>> serviceConfigurations = new HashMap<>();
 
@@ -36,7 +36,8 @@ class AmbariCluster implements ServiceDiscovery.Cluster {
 
     AmbariCluster(String name) {
         this.name = name;
-        components = new HashMap<String, AmbariComponent>();
+        components = new HashMap<>();
+        urlCreator = new AmbariDynamicServiceURLCreator(this);
     }
 
     void addServiceConfiguration(String serviceName, String configurationType, ServiceConfiguration serviceConfig) {
@@ -81,7 +82,7 @@ class AmbariCluster implements ServiceDiscovery.Cluster {
     @Override
     public List<String> getServiceURLs(String serviceName) {
         List<String> urls = new ArrayList<>();
-        urls.addAll(urlCreator.create(this, serviceName));
+        urls.addAll(urlCreator.create(serviceName));
         return urls;
     }
 

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariComponent.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariComponent.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariComponent.java
index 55257fb..d9d5b03 100644
--- a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariComponent.java
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariComponent.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.gateway.topology.discovery.ambari;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -27,7 +28,7 @@ class AmbariComponent {
     private String name        = null;
     private String version     = null;
 
-    private List<String> hostNames = null;
+    private List<String> hostNames = new ArrayList<>();
 
     private Map<String, String> properties = null;
 
@@ -41,35 +42,43 @@ class AmbariComponent {
         this.serviceName = service;
         this.clusterName = cluster;
         this.version = version;
-        this.hostNames = hostNames;
         this.properties = properties;
+
+        if (hostNames != null) {
+            // Add the hostnames individually to prevent adding any null values
+            for (String hostName : hostNames) {
+                if (hostName != null) {
+                    this.hostNames.add(hostName);
+                }
+            }
+        }
     }
 
-    public String getVersion() {
+    String getVersion() {
         return version;
     }
 
-    public String getName() {
+    String getName() {
         return name;
     }
 
-    public String getServiceName() {
+    String getServiceName() {
         return serviceName;
     }
 
-    public String getClusterName() {
+    String getClusterName() {
         return clusterName;
     }
 
-    public List<String> getHostNames() {
+    List<String> getHostNames() {
         return hostNames;
     }
 
-    public Map<String, String> getConfigProperties() {
+    Map<String, String> getConfigProperties() {
         return properties;
     }
 
-    public String getConfigProperty(String propertyName) {
+    String getConfigProperty(String propertyName) {
         return properties.get(propertyName);
     }
 

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariDynamicServiceURLCreator.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariDynamicServiceURLCreator.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariDynamicServiceURLCreator.java
new file mode 100644
index 0000000..ed5d3e7
--- /dev/null
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariDynamicServiceURLCreator.java
@@ -0,0 +1,151 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.gateway.topology.discovery.ambari;
+
+import org.apache.hadoop.gateway.i18n.messages.MessagesFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+class AmbariDynamicServiceURLCreator {
+
+    static final String MAPPING_CONFIG_OVERRIDE_PROPERTY = "org.apache.gateway.topology.discovery.ambari.config";
+
+    private AmbariServiceDiscoveryMessages log = MessagesFactory.get(AmbariServiceDiscoveryMessages.class);
+
+    private AmbariCluster cluster = null;
+    private ServiceURLPropertyConfig config;
+
+    AmbariDynamicServiceURLCreator(AmbariCluster cluster) {
+        this.cluster = cluster;
+
+        String mappingConfiguration = System.getProperty(MAPPING_CONFIG_OVERRIDE_PROPERTY);
+        if (mappingConfiguration != null) {
+            File mappingConfigFile = new File(mappingConfiguration);
+            if (mappingConfigFile.exists()) {
+                try {
+                    config = new ServiceURLPropertyConfig(mappingConfigFile);
+                    log.loadedComponentConfigMappings(mappingConfigFile.getAbsolutePath());
+                } catch (Exception e) {
+                    log.failedToLoadComponentConfigMappings(mappingConfigFile.getAbsolutePath(), e);
+                }
+            }
+        }
+
+        // If there is no valid override configured, fall-back to the internal mapping configuration
+        if (config == null) {
+            config = new ServiceURLPropertyConfig();
+        }
+    }
+
+    AmbariDynamicServiceURLCreator(AmbariCluster cluster, File mappingConfiguration) throws IOException {
+        this.cluster = cluster;
+        config = new ServiceURLPropertyConfig(new FileInputStream(mappingConfiguration));
+    }
+
+    AmbariDynamicServiceURLCreator(AmbariCluster cluster, String mappings) {
+        this.cluster = cluster;
+        config = new ServiceURLPropertyConfig(new ByteArrayInputStream(mappings.getBytes()));
+    }
+
+    List<String> create(String serviceName) {
+        List<String> urls = new ArrayList<>();
+
+        Map<String, String> placeholderValues = new HashMap<>();
+        List<String> componentHostnames = new ArrayList<>();
+        String hostNamePlaceholder = null;
+
+        ServiceURLPropertyConfig.URLPattern pattern = config.getURLPattern(serviceName);
+        if (pattern != null) {
+            for (String propertyName : pattern.getPlaceholders()) {
+                ServiceURLPropertyConfig.Property configProperty = config.getConfigProperty(serviceName, propertyName);
+
+                String propertyValue = null;
+                String propertyType = configProperty.getType();
+                if (ServiceURLPropertyConfig.Property.TYPE_SERVICE.equals(propertyType)) {
+                    log.lookingUpServiceConfigProperty(configProperty.getService(), configProperty.getServiceConfig(), configProperty.getValue());
+                    AmbariCluster.ServiceConfiguration svcConfig =
+                        cluster.getServiceConfiguration(configProperty.getService(), configProperty.getServiceConfig());
+                    if (svcConfig != null) {
+                        propertyValue = svcConfig.getProperties().get(configProperty.getValue());
+                    }
+                } else if (ServiceURLPropertyConfig.Property.TYPE_COMPONENT.equals(propertyType)) {
+                    String compName = configProperty.getComponent();
+                    if (compName != null) {
+                        AmbariComponent component = cluster.getComponent(compName);
+                        if (component != null) {
+                            if (ServiceURLPropertyConfig.Property.PROP_COMP_HOSTNAME.equals(configProperty.getValue())) {
+                                log.lookingUpComponentHosts(compName);
+                                componentHostnames.addAll(component.getHostNames());
+                                hostNamePlaceholder = propertyName; // Remember the host name placeholder
+                            } else {
+                                log.lookingUpComponentConfigProperty(compName, configProperty.getValue());
+                                propertyValue = component.getConfigProperty(configProperty.getValue());
+                            }
+                        }
+                    }
+                } else { // Derived property
+                    log.handlingDerivedProperty(serviceName, configProperty.getType(), configProperty.getName());
+                    ServiceURLPropertyConfig.Property p = config.getConfigProperty(serviceName, configProperty.getName());
+                    propertyValue = p.getValue();
+                    if (propertyValue == null) {
+                        if (p.getConditionHandler() != null) {
+                            propertyValue = p.getConditionHandler().evaluate(config, cluster);
+                        }
+                    }
+                }
+
+                log.determinedPropertyValue(configProperty.getName(), propertyValue);
+                placeholderValues.put(configProperty.getName(), propertyValue);
+            }
+
+            // For patterns with a placeholder value for the hostname (e.g., multiple URL scenarios)
+            if (!componentHostnames.isEmpty()) {
+                for (String componentHostname : componentHostnames) {
+                    String url = pattern.get().replace("{" + hostNamePlaceholder + "}", componentHostname);
+                    urls.add(createURL(url, placeholderValues));
+                }
+            } else { // Single URL result case
+                urls.add(createURL(pattern.get(), placeholderValues));
+            }
+        }
+
+        return urls;
+    }
+
+    private String createURL(String pattern, Map<String, String> placeholderValues) {
+        String url = null;
+        if (pattern != null) {
+            url = pattern;
+            for (String placeHolder : placeholderValues.keySet()) {
+                String value = placeholderValues.get(placeHolder);
+                if (value != null) {
+                    url = url.replace("{" + placeHolder + "}", value);
+                }
+            }
+        }
+        return url;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscovery.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscovery.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscovery.java
index 34f20a7..37f68ae 100644
--- a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscovery.java
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscovery.java
@@ -16,6 +16,13 @@
  */
 package org.apache.hadoop.gateway.topology.discovery.ambari;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
 import net.minidev.json.JSONArray;
 import net.minidev.json.JSONObject;
 import net.minidev.json.JSONValue;
@@ -34,9 +41,6 @@ import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.message.BasicHeader;
 import org.apache.http.util.EntityUtils;
 
-import java.io.IOException;
-import java.util.*;
-
 
 class AmbariServiceDiscovery implements ServiceDiscovery {
 
@@ -50,31 +54,33 @@ class AmbariServiceDiscovery implements ServiceDiscovery {
     static final String AMBARI_SERVICECONFIGS_URI =
             AMBARI_CLUSTERS_URI + "/%s/configurations/service_config_versions?is_current=true";
 
+    private static final String COMPONENT_CONFIG_MAPPING_FILE =
+                                                        "ambari-service-discovery-component-config-mapping.properties";
+
+    private static final AmbariServiceDiscoveryMessages log = MessagesFactory.get(AmbariServiceDiscoveryMessages.class);
+
     // Map of component names to service configuration types
     private static Map<String, String> componentServiceConfigs = new HashMap<>();
     static {
-        componentServiceConfigs.put("NAMENODE", "hdfs-site");
-        componentServiceConfigs.put("RESOURCEMANAGER", "yarn-site");
-        componentServiceConfigs.put("OOZIE_SERVER", "oozie-site");
-        componentServiceConfigs.put("HIVE_SERVER", "hive-site");
-        componentServiceConfigs.put("WEBHCAT_SERVER", "webhcat-site");
-        componentServiceConfigs.put("HBASE_MASTER", "hbase-site");
-    } // TODO: Are there other service components, for which the endpoints can be discovered via Ambari?
+        try {
+            Properties configMapping = new Properties();
+            configMapping.load(AmbariServiceDiscovery.class.getClassLoader().getResourceAsStream(COMPONENT_CONFIG_MAPPING_FILE));
+            for (String componentName : configMapping.stringPropertyNames()) {
+                componentServiceConfigs.put(componentName, configMapping.getProperty(componentName));
+            }
+        } catch (Exception e) {
+            log.failedToLoadServiceDiscoveryConfiguration(COMPONENT_CONFIG_MAPPING_FILE, e);
+        }
+    }
 
     private static final String DEFAULT_USER_ALIAS = "ambari.discovery.user";
     private static final String DEFAULT_PWD_ALIAS  = "ambari.discovery.password";
 
-    private static AmbariServiceURLCreator urlCreator = new AmbariServiceURLCreator();
-
-    private AmbariServiceDiscoveryMessages log = MessagesFactory.get(AmbariServiceDiscoveryMessages.class);
-
     @GatewayService
     private AliasService aliasService;
 
     private CloseableHttpClient httpClient = null;
 
-    private Map<String, Map<String, String>> serviceConfiguration = new HashMap<>();
-
 
     AmbariServiceDiscovery() {
         httpClient = org.apache.http.impl.client.HttpClients.createDefault();
@@ -141,13 +147,21 @@ class AmbariServiceDiscovery implements ServiceDiscovery {
 
                         serviceComponents.put(componentName, serviceName);
 
-//                    String hostName = (String) hostRoles.get("host_name");
-                        String hostName = (String) hostRoles.get("public_host_name"); // Assuming public host name is most applicable
-                        log.discoveredServiceHost(serviceName, hostName);
-                        if (!componentHostNames.containsKey(componentName)) {
-                            componentHostNames.put(componentName, new ArrayList<String>());
+                        // Assuming public host name is more applicable than host_name
+                        String hostName = (String) hostRoles.get("public_host_name");
+                        if (hostName == null) {
+                            // Some (even slightly) older versions of Ambari/HDP do not return public_host_name,
+                            // so fall back to host_name in those cases.
+                            hostName = (String) hostRoles.get("host_name");
+                        }
+
+                        if (hostName != null) {
+                            log.discoveredServiceHost(serviceName, hostName);
+                            if (!componentHostNames.containsKey(componentName)) {
+                                componentHostNames.put(componentName, new ArrayList<String>());
+                            }
+                            componentHostNames.get(componentName).add(hostName);
                         }
-                        componentHostNames.get(componentName).add(hostName);
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscoveryMessages.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscoveryMessages.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscoveryMessages.java
index caa16ed..0661224 100644
--- a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscoveryMessages.java
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceDiscoveryMessages.java
@@ -21,61 +21,101 @@ import org.apache.hadoop.gateway.i18n.messages.MessageLevel;
 import org.apache.hadoop.gateway.i18n.messages.Messages;
 import org.apache.hadoop.gateway.i18n.messages.StackTrace;
 
-@Messages(logger="org.apache.gateway.topology.discovery.ambari")
+@Messages(logger="org.apache.hadoop.gateway.topology.discovery.ambari")
 public interface AmbariServiceDiscoveryMessages {
 
     @Message(level = MessageLevel.ERROR,
-            text = "Encountered an error during cluster {0} discovery: {1}")
+            text = "Failed to load service discovery configuration: {1}")
+    void failedToLoadServiceDiscoveryConfiguration(@StackTrace(level = MessageLevel.ERROR) Exception e);
+
+    @Message(level = MessageLevel.ERROR,
+             text = "Failed to load service discovery configuration {0}: {1}")
+    void failedToLoadServiceDiscoveryConfiguration(final String configuration,
+                               @StackTrace(level = MessageLevel.ERROR) Exception e);
+
+    @Message(level = MessageLevel.ERROR,
+             text = "Encountered an error during cluster {0} discovery: {1}")
     void clusterDiscoveryError(final String clusterName,
                                @StackTrace(level = MessageLevel.ERROR) Exception e);
 
 
     @Message(level = MessageLevel.DEBUG,
-            text = "REST invocation {0} failed: {1}")
+             text = "REST invocation {0} failed: {1}")
     void restInvocationError(final String url,
                              @StackTrace(level = MessageLevel.ERROR) Exception e);
 
 
     @Message(level = MessageLevel.ERROR,
-            text = "Encountered an error attempting to determine the user for alias {0} : {1}")
+             text = "Encountered an error attempting to determine the user for alias {0} : {1}")
     void aliasServiceUserError(final String alias, final String error);
 
 
     @Message(level = MessageLevel.ERROR,
-            text = "Encountered an error attempting to determine the password for alias {0} : {1}")
+             text = "Encountered an error attempting to determine the password for alias {0} : {1}")
     void aliasServicePasswordError(final String alias, final String error);
 
 
     @Message(level = MessageLevel.ERROR,
-            text = "No user configured for Ambari service discovery.")
+             text = "No user configured for Ambari service discovery.")
     void aliasServiceUserNotFound();
 
 
     @Message(level = MessageLevel.ERROR,
-            text = "No password configured for Ambari service discovery.")
+             text = "No password configured for Ambari service discovery.")
     void aliasServicePasswordNotFound();
 
 
     @Message(level = MessageLevel.ERROR,
-            text = "Unexpected REST invocation response code for {0} : {1}")
+             text = "Unexpected REST invocation response code for {0} : {1}")
     void unexpectedRestResponseStatusCode(final String url, int responseStatusCode);
 
 
     @Message(level = MessageLevel.ERROR,
-            text = "REST invocation {0} yielded a response without any JSON.")
+             text = "REST invocation {0} yielded a response without any JSON.")
     void noJSON(final String url);
 
 
     @Message(level = MessageLevel.DEBUG,
-            text = "REST invocation result: {0}")
+             text = "REST invocation result: {0}")
     void debugJSON(final String json);
 
+    @Message(level = MessageLevel.DEBUG,
+            text = "Loaded component configuration mappings: {0}")
+    void loadedComponentConfigMappings(final String mappings);
 
-    @Message(level = MessageLevel.INFO,
-            text = "Discovered: Service: {0}, Host: {1}")
+    @Message(level = MessageLevel.ERROR,
+             text = "Failed to load component configuration property mappings {0}: {1}")
+    void failedToLoadComponentConfigMappings(final String mappings,
+                                             @StackTrace(level = MessageLevel.ERROR) Exception e);
+
+    @Message(level = MessageLevel.DEBUG,
+             text = "Discovered: Service: {0}, Host: {1}")
     void discoveredServiceHost(final String serviceName, final String hostName);
 
 
+    @Message(level = MessageLevel.DEBUG,
+             text = "Querying the cluster for the {0} configuration ({1}) property: {2}")
+    void lookingUpServiceConfigProperty(final String serviceName, final String configType, final String propertyName);
+
+
+    @Message(level = MessageLevel.DEBUG,
+             text = "Querying the cluster for the {0} component configuration property: {1}")
+    void lookingUpComponentConfigProperty(final String componentName, final String propertyName);
+
+
+    @Message(level = MessageLevel.DEBUG,
+             text = "Querying the cluster for the {0} component's hosts")
+    void lookingUpComponentHosts(final String componentName);
+
+
+    @Message(level = MessageLevel.DEBUG,
+            text = "Handling a derived service URL mapping property for the {0} service: type = {1}, name = {2}")
+    void handlingDerivedProperty(final String serviceName, final String propertyType, final String propertyName);
+
+
+    @Message(level = MessageLevel.DEBUG,
+            text = "Determined the service URL mapping property {0} value: {1}")
+    void determinedPropertyValue(final String propertyName, final String propertyValue);
 
 
 }

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceURLCreator.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceURLCreator.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceURLCreator.java
deleted file mode 100644
index 0674642..0000000
--- a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/AmbariServiceURLCreator.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/**
- * 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.gateway.topology.discovery.ambari;
-
-
-import java.util.ArrayList;
-import java.util.List;
-
-class AmbariServiceURLCreator {
-
-    private static final String NAMENODE_SERVICE        = "NAMENODE";
-    private static final String JOBTRACKER_SERVICE      = "JOBTRACKER";
-    private static final String WEBHDFS_SERVICE         = "WEBHDFS";
-    private static final String WEBHCAT_SERVICE         = "WEBHCAT";
-    private static final String OOZIE_SERVICE           = "OOZIE";
-    private static final String WEBHBASE_SERVICE        = "WEBHBASE";
-    private static final String HIVE_SERVICE            = "HIVE";
-    private static final String RESOURCEMANAGER_SERVICE = "RESOURCEMANAGER";
-
-
-    /**
-     * Derive the endpoint URL(s) for the specified service, based on the info from the specified Cluster.
-     *
-     * @param cluster The cluster discovery results
-     * @param serviceName The name of a Hadoop service
-     *
-     * @return One or more endpoint URLs for the specified service.
-     */
-    public List<String> create(AmbariCluster cluster, String serviceName) {
-        List<String> result = null;
-
-        if (NAMENODE_SERVICE.equals(serviceName)) {
-            result = createNameNodeURL(cluster);
-        } else if (JOBTRACKER_SERVICE.equals(serviceName)) {
-            result = createJobTrackerURL(cluster);
-        } else if (WEBHDFS_SERVICE.equals(serviceName)) {
-            result = createWebHDFSURL(cluster);
-        } else if (WEBHCAT_SERVICE.equals(serviceName)) {
-            result = createWebHCatURL(cluster);
-        } else if (OOZIE_SERVICE.equals(serviceName)) {
-            result = createOozieURL(cluster);
-        } else if (WEBHBASE_SERVICE.equals(serviceName)) {
-            result = createWebHBaseURL(cluster);
-        } else if (HIVE_SERVICE.equals(serviceName)) {
-            result = createHiveURL(cluster);
-        } else if (RESOURCEMANAGER_SERVICE.equals(serviceName)) {
-            result = createResourceManagerURL(cluster);
-        }
-
-        return result;
-    }
-
-
-    private List<String> createNameNodeURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariComponent comp = cluster.getComponent("NAMENODE");
-        if (comp != null) {
-            result.add("hdfs://" + comp.getConfigProperty("dfs.namenode.rpc-address"));
-        }
-
-        return result;
-    }
-
-
-    private List<String> createJobTrackerURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariComponent comp = cluster.getComponent("RESOURCEMANAGER");
-        if (comp != null) {
-            result.add("rpc://" + comp.getConfigProperty("yarn.resourcemanager.address"));
-        }
-
-        return result;
-    }
-
-
-    private List<String> createWebHDFSURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariCluster.ServiceConfiguration sc = cluster.getServiceConfiguration("HDFS", "hdfs-site");
-        if (sc != null) {
-            String address = sc.getProperties().get("dfs.namenode.http-address");
-            result.add("http://" + address + "/webhdfs");
-        }
-
-        return result;
-    }
-
-
-    private List<String> createWebHCatURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariComponent webhcat = cluster.getComponent("WEBHCAT_SERVER");
-        if (webhcat != null) {
-            String port = webhcat.getConfigProperty("templeton.port");
-            String host = webhcat.getHostNames().get(0);
-
-            result.add("http://" + host + ":" + port + "/templeton");
-        }
-        return result;
-    }
-
-
-    private List<String> createOozieURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariComponent comp = cluster.getComponent("OOZIE_SERVER");
-        if (comp != null) {
-            result.add(comp.getConfigProperty("oozie.base.url"));
-        }
-
-        return result;
-    }
-
-
-    private List<String> createWebHBaseURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariComponent comp = cluster.getComponent("HBASE_MASTER");
-        if (comp != null) {
-            for (String host : comp.getHostNames()) {
-                result.add("http://" + host + ":60080");
-            }
-        }
-
-        return result;
-    }
-
-
-    private List<String> createHiveURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariComponent hive = cluster.getComponent("HIVE_SERVER");
-        if (hive != null) {
-            String path = hive.getConfigProperty("hive.server2.thrift.http.path");
-            String port = hive.getConfigProperty("hive.server2.thrift.http.port");
-            String transport = hive.getConfigProperty("hive.server2.transport.mode");
-            String useSSL = hive.getConfigProperty("hive.server2.use.SSL");
-            String host = hive.getHostNames().get(0);
-
-            String scheme = null; // What is the scheme for the binary transport mode?
-            if ("http".equals(transport)) {
-                scheme = Boolean.valueOf(useSSL) ? "https" : "http";
-            }
-
-            result.add(scheme + "://" + host + ":" + port + "/" + path);
-        }
-        return result;
-    }
-
-
-    private List<String> createResourceManagerURL(AmbariCluster cluster) {
-        List<String> result = new ArrayList<>();
-
-        AmbariComponent resMan = cluster.getComponent("RESOURCEMANAGER");
-        if (resMan != null) {
-            String webappAddress = resMan.getConfigProperty("yarn.resourcemanager.webapp.address");
-            String httpPolicy = resMan.getConfigProperty("yarn.http.policy");
-            String scheme = ("HTTPS_ONLY".equalsIgnoreCase(httpPolicy)) ? "https" : "http";
-
-            result.add(scheme + "://" + webappAddress + "/ws");
-        }
-
-        return result;
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ConditionalValueHandler.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ConditionalValueHandler.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ConditionalValueHandler.java
new file mode 100644
index 0000000..d76a161
--- /dev/null
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ConditionalValueHandler.java
@@ -0,0 +1,24 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.gateway.topology.discovery.ambari;
+
+
+interface ConditionalValueHandler {
+
+    String evaluate(ServiceURLPropertyConfig config, AmbariCluster cluster);
+
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/PropertyEqualsHandler.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/PropertyEqualsHandler.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/PropertyEqualsHandler.java
new file mode 100644
index 0000000..642a676
--- /dev/null
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/PropertyEqualsHandler.java
@@ -0,0 +1,76 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.gateway.topology.discovery.ambari;
+
+
+class PropertyEqualsHandler implements ConditionalValueHandler {
+
+    private String serviceName                        = null;
+    private String propertyName                       = null;
+    private String propertyValue                      = null;
+    private ConditionalValueHandler affirmativeResult = null;
+    private ConditionalValueHandler negativeResult    = null;
+
+    PropertyEqualsHandler(String                  serviceName,
+                          String                  propertyName,
+                          String                  propertyValue,
+                          ConditionalValueHandler affirmativeResult,
+                          ConditionalValueHandler negativeResult) {
+        this.serviceName       = serviceName;
+        this.propertyName      = propertyName;
+        this.propertyValue     = propertyValue;
+        this.affirmativeResult = affirmativeResult;
+        this.negativeResult    = negativeResult;
+    }
+
+    @Override
+    public String evaluate(ServiceURLPropertyConfig config, AmbariCluster cluster) {
+        String result = null;
+
+        ServiceURLPropertyConfig.Property p = config.getConfigProperty(serviceName, propertyName);
+        if (p != null) {
+            String value = getActualPropertyValue(cluster, p);
+            if (propertyValue.equals(value)) {
+                result = affirmativeResult.evaluate(config, cluster);
+            } else if (negativeResult != null) {
+                result = negativeResult.evaluate(config, cluster);
+            }
+
+            // Check if the result is a reference to a local derived property
+            ServiceURLPropertyConfig.Property derived = config.getConfigProperty(serviceName, result);
+            if (derived != null) {
+                result = getActualPropertyValue(cluster, derived);
+            }
+        }
+
+        return result;
+    }
+
+    private String getActualPropertyValue(AmbariCluster cluster, ServiceURLPropertyConfig.Property property) {
+        String value = null;
+        String propertyType = property.getType();
+        if (ServiceURLPropertyConfig.Property.TYPE_COMPONENT.equals(propertyType)) {
+            AmbariComponent component = cluster.getComponent(property.getComponent());
+            if (component != null) {
+                value = component.getConfigProperty(property.getValue());
+            }
+        } else if (ServiceURLPropertyConfig.Property.TYPE_SERVICE.equals(propertyType)) {
+            value = cluster.getServiceConfiguration(property.getService(), property.getServiceConfig()).getProperties().get(property.getValue());
+        }
+        return value;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ServiceURLPropertyConfig.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ServiceURLPropertyConfig.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ServiceURLPropertyConfig.java
new file mode 100644
index 0000000..3330cc3
--- /dev/null
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/ServiceURLPropertyConfig.java
@@ -0,0 +1,324 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.gateway.topology.discovery.ambari;
+
+import org.apache.hadoop.gateway.i18n.messages.MessagesFactory;
+import org.apache.hadoop.gateway.util.XmlUtils;
+import org.w3c.dom.Document;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import javax.xml.xpath.XPath;
+import javax.xml.xpath.XPathConstants;
+import javax.xml.xpath.XPathExpression;
+import javax.xml.xpath.XPathExpressionException;
+import javax.xml.xpath.XPathFactory;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Service URL pattern mapping configuration model.
+ */
+class ServiceURLPropertyConfig {
+
+    private static final AmbariServiceDiscoveryMessages log = MessagesFactory.get(AmbariServiceDiscoveryMessages.class);
+
+    private static final String ATTR_NAME = "name";
+
+    private static XPathExpression SERVICE_URL_PATTERN_MAPPINGS;
+    private static XPathExpression URL_PATTERN;
+    private static XPathExpression PROPERTIES;
+    static {
+        XPath xpath = XPathFactory.newInstance().newXPath();
+        try {
+            SERVICE_URL_PATTERN_MAPPINGS = xpath.compile("/service-discovery-url-mappings/service");
+            URL_PATTERN                  = xpath.compile("url-pattern/text()");
+            PROPERTIES                   = xpath.compile("properties/property");
+        } catch (XPathExpressionException e) {
+            e.printStackTrace();
+        }
+    }
+
+    private static final String DEFAULT_SERVICE_URL_MAPPINGS = "ambari-service-discovery-url-mappings.xml";
+
+    private Map<String, URLPattern> urlPatterns = new HashMap<>();
+
+    private Map<String, Map<String, Property>> properties = new HashMap<>();
+
+
+    /**
+     * The default service URL pattern to property mapping configuration will be used.
+     */
+    ServiceURLPropertyConfig() {
+        this(ServiceURLPropertyConfig.class.getClassLoader().getResourceAsStream(DEFAULT_SERVICE_URL_MAPPINGS));
+    }
+
+    /**
+     * The default service URL pattern to property mapping configuration will be used.
+     */
+    ServiceURLPropertyConfig(File mappingConfigurationFile) throws Exception {
+        this(new FileInputStream(mappingConfigurationFile));
+    }
+
+    /**
+     *
+     * @param source An InputStream for the XML content
+     */
+    ServiceURLPropertyConfig(InputStream source) {
+        // Parse the XML, and build the model
+        try {
+            Document doc = XmlUtils.readXml(source);
+
+            NodeList serviceNodes =
+                    (NodeList) SERVICE_URL_PATTERN_MAPPINGS.evaluate(doc, XPathConstants.NODESET);
+            for (int i=0; i < serviceNodes.getLength(); i++) {
+                Node serviceNode = serviceNodes.item(i);
+                String serviceName = serviceNode.getAttributes().getNamedItem(ATTR_NAME).getNodeValue();
+                properties.put(serviceName, new HashMap<String, Property>());
+
+                Node urlPatternNode = (Node) URL_PATTERN.evaluate(serviceNode, XPathConstants.NODE);
+                if (urlPatternNode != null) {
+                    urlPatterns.put(serviceName, new URLPattern(urlPatternNode.getNodeValue()));
+                }
+
+                NodeList propertiesNode = (NodeList) PROPERTIES.evaluate(serviceNode, XPathConstants.NODESET);
+                if (propertiesNode != null) {
+                    processProperties(serviceName, propertiesNode);
+                }
+            }
+        } catch (Exception e) {
+            log.failedToLoadServiceDiscoveryConfiguration(e);
+        } finally {
+            try {
+                source.close();
+            } catch (IOException e) {
+                // Ignore
+            }
+        }
+    }
+
+    private void processProperties(String serviceName, NodeList propertyNodes) {
+        for (int i = 0; i < propertyNodes.getLength(); i++) {
+            Property p = Property.createProperty(serviceName, propertyNodes.item(i));
+            properties.get(serviceName).put(p.getName(), p);
+        }
+    }
+
+    URLPattern getURLPattern(String service) {
+        return urlPatterns.get(service);
+    }
+
+    Property getConfigProperty(String service, String property) {
+        return properties.get(service).get(property);
+    }
+
+    static class URLPattern {
+        String pattern;
+        List<String> placeholders = new ArrayList<>();
+
+        URLPattern(String pattern) {
+            this.pattern = pattern;
+
+            final Pattern regex = Pattern.compile("\\{(.*?)}", Pattern.DOTALL);
+            final Matcher matcher = regex.matcher(pattern);
+            while( matcher.find() ){
+                placeholders.add(matcher.group(1));
+            }
+        }
+
+        String get() {return pattern; }
+        List<String> getPlaceholders() {
+            return placeholders;
+        }
+    }
+
+    static class Property {
+        static final String TYPE_SERVICE   = "SERVICE";
+        static final String TYPE_COMPONENT = "COMPONENT";
+        static final String TYPE_DERIVED   = "DERIVED";
+
+        static final String PROP_COMP_HOSTNAME = "component.host.name";
+
+        static final String ATTR_NAME     = "name";
+        static final String ATTR_PROPERTY = "property";
+        static final String ATTR_VALUE    = "value";
+
+        static XPathExpression HOSTNAME;
+        static XPathExpression SERVICE_CONFIG;
+        static XPathExpression COMPONENT;
+        static XPathExpression CONFIG_PROPERTY;
+        static XPathExpression IF;
+        static XPathExpression THEN;
+        static XPathExpression ELSE;
+        static XPathExpression TEXT;
+        static {
+            XPath xpath = XPathFactory.newInstance().newXPath();
+            try {
+                HOSTNAME        = xpath.compile("hostname");
+                SERVICE_CONFIG  = xpath.compile("service-config");
+                COMPONENT       = xpath.compile("component");
+                CONFIG_PROPERTY = xpath.compile("config-property");
+                IF              = xpath.compile("if");
+                THEN            = xpath.compile("then");
+                ELSE            = xpath.compile("else");
+                TEXT            = xpath.compile("text()");
+            } catch (XPathExpressionException e) {
+                e.printStackTrace();
+            }
+        }
+
+
+        String type;
+        String name;
+        String component;
+        String service;
+        String serviceConfig;
+        String value;
+        ConditionalValueHandler conditionHandler = null;
+
+        private Property(String type,
+                         String propertyName,
+                         String component,
+                         String service,
+                         String configType,
+                         String value,
+                         ConditionalValueHandler pch) {
+            this.type = type;
+            this.name = propertyName;
+            this.service = service;
+            this.component = component;
+            this.serviceConfig = configType;
+            this.value = value;
+            conditionHandler = pch;
+        }
+
+        static Property createProperty(String serviceName, Node propertyNode) {
+            String propertyName = propertyNode.getAttributes().getNamedItem(ATTR_NAME).getNodeValue();
+            String propertyType = null;
+            String serviceType = null;
+            String configType = null;
+            String componentType = null;
+            String value = null;
+            ConditionalValueHandler pch = null;
+
+            try {
+                Node hostNameNode = (Node) HOSTNAME.evaluate(propertyNode, XPathConstants.NODE);
+                if (hostNameNode != null) {
+                    value = PROP_COMP_HOSTNAME;
+                }
+
+                // Check for a service-config node
+                Node scNode = (Node) SERVICE_CONFIG.evaluate(propertyNode, XPathConstants.NODE);
+                if (scNode != null) {
+                    // Service config property
+                    propertyType = Property.TYPE_SERVICE;
+                    serviceType = scNode.getAttributes().getNamedItem(ATTR_NAME).getNodeValue();
+                    Node scTextNode = (Node) TEXT.evaluate(scNode, XPathConstants.NODE);
+                    configType = scTextNode.getNodeValue();
+                } else { // If not service-config node, check for a component config node
+                    Node cNode = (Node) COMPONENT.evaluate(propertyNode, XPathConstants.NODE);
+                    if (cNode != null) {
+                        // Component config property
+                        propertyType = Property.TYPE_COMPONENT;
+                        componentType = cNode.getFirstChild().getNodeValue();
+                        Node cTextNode = (Node) TEXT.evaluate(cNode, XPathConstants.NODE);
+                        configType = cTextNode.getNodeValue();
+                        componentType = cTextNode.getNodeValue();
+                    }
+                }
+
+                // Check for a config property node
+                Node cpNode = (Node) CONFIG_PROPERTY.evaluate(propertyNode, XPathConstants.NODE);
+                if (cpNode != null) {
+                    // Check for a condition element
+                    Node ifNode = (Node) IF.evaluate(cpNode, XPathConstants.NODE);
+                    if (ifNode != null) {
+                        propertyType = TYPE_DERIVED;
+                        pch = getConditionHandler(serviceName, ifNode);
+                    } else {
+                        Node cpTextNode = (Node) TEXT.evaluate(cpNode, XPathConstants.NODE);
+                        value = cpTextNode.getNodeValue();
+                    }
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+
+            // Create and return the property representation
+            return new Property(propertyType, propertyName, componentType, serviceType, configType, value, pch);
+        }
+
+        private static ConditionalValueHandler getConditionHandler(String serviceName, Node ifNode) throws Exception {
+            ConditionalValueHandler result = null;
+
+            if (ifNode != null) {
+                NamedNodeMap attrs = ifNode.getAttributes();
+                String comparisonPropName = attrs.getNamedItem(ATTR_PROPERTY).getNodeValue();
+                String comparisonValue = attrs.getNamedItem(ATTR_VALUE).getNodeValue();
+
+                ConditionalValueHandler affirmativeResult = null;
+                Node thenNode = (Node) THEN.evaluate(ifNode, XPathConstants.NODE);
+                if (thenNode != null) {
+                    Node subIfNode = (Node) IF.evaluate(thenNode, XPathConstants.NODE);
+                    if (subIfNode != null) {
+                        affirmativeResult = getConditionHandler(serviceName, subIfNode);
+                    } else {
+                        affirmativeResult = new SimpleValueHandler(thenNode.getFirstChild().getNodeValue());
+                    }
+                }
+
+                ConditionalValueHandler negativeResult = null;
+                Node elseNode = (Node) ELSE.evaluate(ifNode, XPathConstants.NODE);
+                if (elseNode != null) {
+                    Node subIfNode = (Node) IF.evaluate(elseNode, XPathConstants.NODE);
+                    if (subIfNode != null) {
+                        negativeResult = getConditionHandler(serviceName, subIfNode);
+                    } else {
+                        negativeResult = new SimpleValueHandler(elseNode.getFirstChild().getNodeValue());
+                    }
+                }
+
+                result = new PropertyEqualsHandler(serviceName,
+                        comparisonPropName,
+                        comparisonValue,
+                        affirmativeResult,
+                        negativeResult);
+            }
+
+            return result;
+        }
+
+        String getType() { return type; }
+        String getName() { return name; }
+        String getComponent() { return component; }
+        String getService() { return service; }
+        String getServiceConfig() { return serviceConfig; }
+        String getValue() {
+            return value;
+        }
+        ConditionalValueHandler getConditionHandler() { return conditionHandler; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/SimpleValueHandler.java
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/SimpleValueHandler.java b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/SimpleValueHandler.java
new file mode 100644
index 0000000..8e0cd75
--- /dev/null
+++ b/gateway-discovery-ambari/src/main/java/org/apache/hadoop/gateway/topology/discovery/ambari/SimpleValueHandler.java
@@ -0,0 +1,32 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.gateway.topology.discovery.ambari;
+
+
+class SimpleValueHandler implements ConditionalValueHandler {
+    private String value;
+
+    SimpleValueHandler(String value) {
+        this.value = value;
+    }
+
+    @Override
+    public String evaluate(ServiceURLPropertyConfig config, AmbariCluster cluster) {
+        return value;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-component-config-mapping.properties
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-component-config-mapping.properties b/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-component-config-mapping.properties
new file mode 100644
index 0000000..a48b28c
--- /dev/null
+++ b/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-component-config-mapping.properties
@@ -0,0 +1,36 @@
+##########################################################################
+# 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.
+##########################################################################
+
+##########################################################################
+# Hadoop service component configuration mappings
+# Service component name = service component configuration type
+##########################################################################
+NAMENODE=hdfs-site
+RESOURCEMANAGER=yarn-site
+OOZIE_SERVER=oozie-site
+HIVE_SERVER=hive-site
+WEBHCAT_SERVER=webhcat-site
+HBASE_MASTER=hbase-site
+DRUID_COORDINATOR=druid-coordinator
+DRUID_BROKER=druid-broker
+DRUID_ROUTER=druid-router
+DRUID_OVERLORD=druid-overlord
+DRUID_SUPERSET=druid-superset
+ATLAS_SERVER=application-properties
+ZEPPELIN_MASTER=zeppelin-config
+#RANGER=TODO

http://git-wip-us.apache.org/repos/asf/knox/blob/7b401def/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-url-mappings.xml
----------------------------------------------------------------------
diff --git a/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-url-mappings.xml b/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-url-mappings.xml
new file mode 100644
index 0000000..8953b8d
--- /dev/null
+++ b/gateway-discovery-ambari/src/main/resources/ambari-service-discovery-url-mappings.xml
@@ -0,0 +1,398 @@
+<?xml version="1.0" encoding="utf-8"?>
+<!--
+  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.
+-->
+<!-- ==================================================================== -->
+<!--  Externalized mapping of Hadoop service URL patterns and the Ambari  -->
+<!--  ServiceDiscovery model properties. This configuration is used by    -->
+<!--  the Ambari ServiceDiscovery implementation to construct service     -->
+<!--  URLs suitable for use in a Knox topology file.                      -->
+<!-- ==================================================================== -->
+<service-discovery-url-mappings>
+
+    <service name="NAMENODE">
+        <url-pattern>hdfs://{DFS_NAMENODE_RPC_ADDRESS}</url-pattern>
+        <properties>
+            <property name="DFS_NAMENODE_RPC_ADDRESS">
+                <component>NAMENODE</component>
+                <config-property>dfs.namenode.rpc-address</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="JOBTRACKER">
+        <url-pattern>rpc://{YARN_RM_ADDRESS}</url-pattern>
+        <properties>
+            <property name="YARN_RM_ADDRESS">
+                <component>RESOURCEMANAGER</component>
+                <config-property>yarn.resourcemanager.address</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="WEBHDFS">
+        <url-pattern>http://{WEBHDFS_ADDRESS}/webhdfs</url-pattern>
+        <properties>
+            <property name="WEBHDFS_ADDRESS">
+                <service-config name="HDFS">hdfs-site</service-config>
+                <config-property>dfs.namenode.http-address</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="WEBHCAT">
+        <url-pattern>http://{HOST}:{PORT}/templeton</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>WEBHCAT_SERVER</component>
+                <hostname/>
+            </property>
+            <property name="PORT">
+                <component>WEBHCAT_SERVER</component>
+                <config-property>templeton.port</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="OOZIE">
+        <url-pattern>{OOZIE_ADDRESS}</url-pattern>
+        <properties>
+            <property name="OOZIE_ADDRESS">
+                <component>OOZIE_SERVER</component>
+                <config-property>oozie.base.url</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="WEBHBASE">
+        <url-pattern>http://{HOST}:60080</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>HBASE_MASTER</component>
+                <hostname/>
+            </property>
+        </properties>
+    </service>
+
+    <service name="RESOURCEMANAGER">
+        <url-pattern>{SCHEME}://{WEBAPP_ADDRESS}/ws</url-pattern>
+        <properties>
+            <property name="WEBAPP_HTTP_ADDRESS">
+                <component>RESOURCEMANAGER</component>
+                <config-property>yarn.resourcemanager.webapp.address</config-property>
+            </property>
+            <property name="WEBAPP_HTTPS_ADDRESS">
+                <component>RESOURCEMANAGER</component>
+                <config-property>yarn.resourcemanager.webapp.https.address</config-property>
+            </property>
+            <property name="HTTP_POLICY">
+                <component>RESOURCEMANAGER</component>
+                <config-property>yarn.http.policy</config-property>
+            </property>
+            <property name="SCHEME">
+                <config-property>
+                    <if property="HTTP_POLICY" value="HTTPS_ONLY">
+                        <then>https</then>
+                        <else>http</else>
+                    </if>
+                </config-property>
+            </property>
+            <property name="WEBAPP_ADDRESS">
+                <config-property>
+                    <if property="HTTP_POLICY" value="HTTPS_ONLY">
+                        <then>WEBAPP_HTTPS_ADDRESS</then>
+                        <else>WEBAPP_HTTP_ADDRESS</else>
+                    </if>
+                </config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="HIVE">
+        <url-pattern>{SCHEME}://{HOST}:{PORT}/{PATH}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>HIVE_SERVER</component>
+                <hostname/>
+            </property>
+            <property name="USE_SSL">
+                <component>HIVE_SERVER</component>
+                <config-property>hive.server2.use.SSL</config-property>
+            </property>
+            <property name="PATH">
+                <component>HIVE_SERVER</component>
+                <config-property>hive.server2.thrift.http.path</config-property>
+            </property>
+            <property name="PORT">
+                <component>HIVE_SERVER</component>
+                <config-property>hive.server2.thrift.http.port</config-property>
+            </property>
+            <property name="SCHEME">
+                 <config-property>
+                    <if property="USE_SSL" value="true">
+                        <then>https</then>
+                        <else>http</else>
+                    </if>
+                </config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="DRUID-COORDINATOR">
+        <url-pattern>http://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>DRUID_COORDINATOR</component>
+                <hostname/>
+            </property>
+            <property name="PORT">
+                <component>DRUID_COORDINATOR</component>
+                <config-property>druid.port</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="DRUID-BROKER">
+        <url-pattern>http://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>DRUID_BROKER</component>
+                <hostname/>
+            </property>
+            <property name="PORT">
+                <component>DRUID_BROKER</component>
+                <config-property>druid.port</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="DRUID-ROUTER">
+        <url-pattern>http://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>DRUID_ROUTER</component>
+                <hostname/>
+            </property>
+            <property name="PORT">
+                <component>DRUID_ROUTER</component>
+                <config-property>druid.port</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="DRUID-OVERLORD">
+        <url-pattern>http://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>DRUID_OVERLORD</component>
+                <hostname/>
+            </property>
+            <property name="PORT">
+                <component>DRUID_OVERLORD</component>
+                <config-property>druid.port</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="SUPERSET">
+        <url-pattern>http://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>DRUID_SUPERSET</component>
+                <hostname/>
+            </property>
+            <property name="PORT">
+                <component>DRUID_SUPERSET</component>
+                <config-property>SUPERSET_WEBSERVER_PORT</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="ATLAS-API">
+        <url-pattern>{REST_ADDRESS}</url-pattern>
+        <properties>
+            <property name="REST_ADDRESS">
+                <component>ATLAS_SERVER</component>
+                <config-property>atlas.rest.address</config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="ATLAS">
+        <url-pattern>{SCHEME}://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>ATLAS_SERVER</component>
+                <hostname/>
+            </property>
+            <property name="TLS_ENABLED">
+                <component>ATLAS_SERVER</component>
+                <config-property>atlas.enableTLS</config-property>
+            </property>
+            <property name="HTTP_PORT">
+                <component>ATLAS_SERVER</component>
+                <config-property>atlas.server.http.port</config-property>
+            </property>
+            <property name="HTTPS_PORT">
+                <component>ATLAS_SERVER</component>
+                <config-property>atlas.server.https.port</config-property>
+            </property>
+            <property name="PORT">
+                <config-property>
+                    <if property="TLS_ENABLED" value="true">
+                        <then>HTTPS_PORT</then>
+                        <else>HTTP_PORT</else>
+                    </if>
+                </config-property>
+            </property>
+            <property name="SCHEME">
+                <config-property>
+                    <if property="TLS_ENABLED" value="true">
+                        <then>https</then>
+                        <else>http</else>
+                    </if>
+                </config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="ZEPPELIN">
+        <url-pattern>{SCHEME}://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>ZEPPELIN_MASTER</component>
+                <hostname/>
+            </property>
+            <property name="SSL">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.ssl</config-property>
+            </property>
+            <property name="SCHEME">
+                <config-property>
+                    <if property="SSL" value="true">
+                        <then>https</then>
+                        <else>http</else>
+                    </if>
+                </config-property>
+            </property>
+            <property name="HTTPS_PORT">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.server.ssl.port</config-property>
+            </property>
+            <property name="HTTP_PORT">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.server.port</config-property>
+            </property>
+            <property name="PORT">
+                <config-property>
+                    <if property="SSL" value="true">
+                        <then>HTTPS_PORT</then>
+                        <else>HTTP_PORT</else>
+                    </if>
+                </config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="ZEPPELINUI">
+        <url-pattern>{SCHEME}://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>ZEPPELIN_MASTER</component>
+                <hostname/>
+            </property>
+            <property name="SSL">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.ssl</config-property>
+            </property>
+            <property name="SCHEME">
+                <config-property>
+                    <if property="SSL" value="true">
+                        <then>https</then>
+                        <else>http</else>
+                    </if>
+                </config-property>
+            </property>
+            <property name="HTTPS_PORT">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.server.ssl.port</config-property>
+            </property>
+            <property name="HTTP_PORT">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.server.port</config-property>
+            </property>
+            <property name="PORT">
+                <config-property>
+                    <if property="SSL" value="true">
+                        <then>HTTPS_PORT</then>
+                        <else>HTTP_PORT</else>
+                    </if>
+                </config-property>
+            </property>
+        </properties>
+    </service>
+
+    <service name="ZEPPELINWS">
+        <url-pattern>{SCHEME}://{HOST}:{PORT}</url-pattern>
+        <properties>
+            <property name="HOST">
+                <component>ZEPPELIN_MASTER</component>
+                <hostname/>
+            </property>
+            <property name="SSL">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.ssl</config-property>
+            </property>
+            <property name="HTTPS_PORT">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.server.ssl.port</config-property>
+            </property>
+            <property name="HTTP_PORT">
+                <component>ZEPPELIN_MASTER</component>
+                <config-property>zeppelin.server.port</config-property>
+            </property>
+            <property name="SCHEME">
+                <config-property>
+                    <if property="SSL" value="true">
+                        <then>wss</then>
+                        <else>ws</else>
+                    </if>
+                </config-property>
+            </property>
+            <property name="PORT">
+                <config-property>
+                    <if property="SSL" value="true">
+                        <then>HTTPS_PORT</then>
+                        <else>HTTP_PORT</else>
+                    </if>
+                </config-property>
+            </property>
+        </properties>
+    </service>
+
+
+<!-- TODO:
+    <service name="YARNUI">
+    </service>
+
+    <service name="RANGER">
+    </service>
+
+    <service name="RANGERUI">
+    </service>
+-->
+
+</service-discovery-url-mappings>