You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by su...@apache.org on 2015/01/06 18:07:57 UTC

knox git commit: KNOX-483 initial commit for loading service definitions

Repository: knox
Updated Branches:
  refs/heads/KNOX-481 a8927c91a -> 6a209042d


KNOX-483 initial commit for loading service definitions


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

Branch: refs/heads/KNOX-481
Commit: 6a209042d7971d2e092acac0c3f77f2cb7021af3
Parents: a8927c9
Author: Sumit Gupta <su...@apache.org>
Authored: Tue Jan 6 12:07:29 2015 -0500
Committer: Sumit Gupta <su...@apache.org>
Committed: Tue Jan 6 12:07:29 2015 -0500

----------------------------------------------------------------------
 gateway-server/pom.xml                          |   8 +
 .../gateway/config/impl/GatewayConfigImpl.java  |   5 +
 .../gateway/deploy/DeploymentFactory.java       |  54 +++---
 .../ServiceDefinitionDeploymentContributor.java | 108 +++++++++++
 .../gateway/util/ServiceDefinitionsLoader.java  |  95 +++++++++
 gateway-service-definitions/pom.xml             |  39 ++++
 .../service/definition/PolicyBinding.java       |  21 ++
 .../service/definition/RewriteFilter.java       |  47 +++++
 .../service/definition/ServiceDefinition.java   |  73 +++++++
 .../gateway/service/definition/UrlBinding.java  |  49 +++++
 .../services/yarn-rm/2.5.0/rewrite.xml          | 193 +++++++++++++++++++
 .../services/yarn-rm/2.5.0/service.xml          |  54 ++++++
 .../definition/ServiceDefinitionTest.java       |  46 +++++
 .../ResourceManagerDeploymentContributor.java   |   9 +-
 .../hadoop/gateway/config/GatewayConfig.java    |   6 +
 .../hadoop/gateway/GatewayFuncTestDriver.java   |  13 ++
 .../hadoop/gateway/GatewayTestConfig.java       |   6 +-
 pom.xml                                         |   6 +
 18 files changed, 805 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-server/pom.xml
----------------------------------------------------------------------
diff --git a/gateway-server/pom.xml b/gateway-server/pom.xml
index 3f736f3..1304cd0 100644
--- a/gateway-server/pom.xml
+++ b/gateway-server/pom.xml
@@ -178,6 +178,14 @@
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.knox</groupId>
+            <artifactId>gateway-service-definitions</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.knox</groupId>
+            <artifactId>gateway-provider-rewrite</artifactId>
+        </dependency>
 
         <!-- ********** ********** ********** ********** ********** ********** -->
         <!-- ********** Test Dependencies                           ********** -->

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-server/src/main/java/org/apache/hadoop/gateway/config/impl/GatewayConfigImpl.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/config/impl/GatewayConfigImpl.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/config/impl/GatewayConfigImpl.java
index d4819c1..8276e82 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/config/impl/GatewayConfigImpl.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/config/impl/GatewayConfigImpl.java
@@ -153,6 +153,11 @@ public class GatewayConfigImpl extends Configuration implements GatewayConfig {
   }
 
   @Override
+  public String getGatewayStacksDir() {
+    return getVar( GATEWAY_DATA_HOME_VAR, getGatewayHomeDir() + File.separator + "stacks" );
+  }
+
+  @Override
   public String getHadoopConfDir() {
     return get( HADOOP_CONF_DIR );
   }

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/DeploymentFactory.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/DeploymentFactory.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/DeploymentFactory.java
index d1069f4..9dfb808 100644
--- a/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/DeploymentFactory.java
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/DeploymentFactory.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.gateway.services.registry.ServiceRegistry;
 import org.apache.hadoop.gateway.topology.Provider;
 import org.apache.hadoop.gateway.topology.Service;
 import org.apache.hadoop.gateway.topology.Topology;
+import org.apache.hadoop.gateway.util.ServiceDefinitionsLoader;
 import org.jboss.shrinkwrap.api.ShrinkWrap;
 import org.jboss.shrinkwrap.api.asset.Asset;
 import org.jboss.shrinkwrap.api.asset.StringAsset;
@@ -40,6 +41,7 @@ import org.jboss.shrinkwrap.descriptor.api.webapp30.WebAppDescriptor;
 import org.jboss.shrinkwrap.descriptor.api.webcommon30.ServletType;
 
 import java.beans.Statement;
+import java.io.File;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.ArrayList;
@@ -55,6 +57,7 @@ import java.util.LinkedHashMap;
 public abstract class DeploymentFactory {
 
   private static final String DEFAULT_APP_REDIRECT_CONTEXT_PATH = "redirectTo";
+  private static final String STACKS_SERVICES_DIRECTORY = "services";
   private static GatewayResources res = ResourcesFactory.get( GatewayResources.class );
   private static GatewayMessages log = MessagesFactory.get( GatewayMessages.class );
   private static GatewayServices gatewayServices = null;
@@ -77,6 +80,12 @@ public abstract class DeploymentFactory {
 
   public static WebArchive createDeployment( GatewayConfig config, Topology topology ) {
     DeploymentContext context = null;
+     //TODO move the loading of service defs
+    String stacks = config.getGatewayStacksDir();
+    File stacksDir = new File(stacks, STACKS_SERVICES_DIRECTORY);
+    Set<ServiceDeploymentContributor> deploymentContributors = ServiceDefinitionsLoader.loadServiceDefinitions(stacksDir);
+    addServiceDeploymentContributors(SERVICE_CONTRIBUTOR_MAP, deploymentContributors.iterator());
+
     Map<String,List<ProviderDeploymentContributor>> providers = selectContextProviders( topology );
     Map<String,List<ServiceDeploymentContributor>> services = selectContextServices( topology );
     context = createDeploymentContext( config, topology.getName(), topology, providers, services );
@@ -398,35 +407,36 @@ public abstract class DeploymentFactory {
   }  
   
   private static void loadServiceContributors() {
-    Set<ServiceDeploymentContributor> set = new HashSet<ServiceDeploymentContributor>();
     Map<String,Map<String,ServiceDeploymentContributor>> roleMap
         = new HashMap<String,Map<String,ServiceDeploymentContributor>>();
 
     ServiceLoader<ServiceDeploymentContributor> loader = ServiceLoader.load( ServiceDeploymentContributor.class );
     Iterator<ServiceDeploymentContributor> contributors = loader.iterator();
-    while( contributors.hasNext() ) {
-      ServiceDeploymentContributor contributor = contributors.next();
-      if( contributor.getName() == null ) {
-        log.ignoringServiceContributorWithMissingName( contributor.getClass().getName() );
-        continue;
-      }
-      if( contributor.getRole() == null ) {
-        log.ignoringServiceContributorWithMissingRole( contributor.getClass().getName() );
-        continue;
-      }
-      set.add( contributor );
-      Map nameMap = roleMap.get( contributor.getRole() );
-      if( nameMap == null ) {
-        nameMap = new HashMap<String,ServiceDeploymentContributor>();
-        roleMap.put( contributor.getRole(), nameMap );
-      }
-      nameMap.put( contributor.getName(), contributor );
-    }
-    //SERVICE_CONTRIBUTORS = set;
-    SERVICE_CONTRIBUTOR_MAP = roleMap;
+     addServiceDeploymentContributors(roleMap, contributors);
+     SERVICE_CONTRIBUTOR_MAP = roleMap;
   }
 
-  private static void loadProviderContributors() {
+   private static void addServiceDeploymentContributors(Map<String, Map<String, ServiceDeploymentContributor>> roleMap, Iterator<ServiceDeploymentContributor> contributors) {
+      while( contributors.hasNext() ) {
+        ServiceDeploymentContributor contributor = contributors.next();
+        if( contributor.getName() == null ) {
+          log.ignoringServiceContributorWithMissingName( contributor.getClass().getName() );
+          continue;
+        }
+        if( contributor.getRole() == null ) {
+          log.ignoringServiceContributorWithMissingRole( contributor.getClass().getName() );
+          continue;
+        }
+        Map nameMap = roleMap.get( contributor.getRole() );
+        if( nameMap == null ) {
+          nameMap = new HashMap<String,ServiceDeploymentContributor>();
+          roleMap.put( contributor.getRole(), nameMap );
+        }
+        nameMap.put( contributor.getName(), contributor );
+      }
+   }
+
+   private static void loadProviderContributors() {
     Set<ProviderDeploymentContributor> set = new HashSet<ProviderDeploymentContributor>();
     Map<String,Map<String,ProviderDeploymentContributor>> roleMap
         = new HashMap<String,Map<String,ProviderDeploymentContributor>>();

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/impl/ServiceDefinitionDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/impl/ServiceDefinitionDeploymentContributor.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/impl/ServiceDefinitionDeploymentContributor.java
new file mode 100644
index 0000000..dd2a4c9
--- /dev/null
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/deploy/impl/ServiceDefinitionDeploymentContributor.java
@@ -0,0 +1,108 @@
+/**
+ * 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.deploy.impl;
+
+import org.apache.hadoop.gateway.deploy.DeploymentContext;
+import org.apache.hadoop.gateway.deploy.ServiceDeploymentContributorBase;
+import org.apache.hadoop.gateway.descriptor.FilterParamDescriptor;
+import org.apache.hadoop.gateway.descriptor.ResourceDescriptor;
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteRulesDescriptor;
+import org.apache.hadoop.gateway.service.definition.RewriteFilter;
+import org.apache.hadoop.gateway.service.definition.ServiceDefinition;
+import org.apache.hadoop.gateway.service.definition.UrlBinding;
+import org.apache.hadoop.gateway.topology.Service;
+
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ServiceDefinitionDeploymentContributor extends ServiceDeploymentContributorBase {
+
+    private ServiceDefinition serviceDefinition;
+
+    private UrlRewriteRulesDescriptor serviceRules;
+
+    public ServiceDefinitionDeploymentContributor(ServiceDefinition serviceDefinition, UrlRewriteRulesDescriptor serviceRules) {
+        this.serviceDefinition = serviceDefinition;
+        this.serviceRules = serviceRules;
+    }
+
+    @Override
+    public String getRole() {
+        return serviceDefinition.getRole();
+    }
+
+    @Override
+    public String getName() {
+        return serviceDefinition.getName();
+    }
+
+    @Override
+    public void contributeService(DeploymentContext context, Service service) throws Exception {
+        System.out.println("contributing service def");
+        contributeRewriteRules(context, service);
+        contributeResources(context, service);
+    }
+
+    private void contributeRewriteRules(DeploymentContext context, Service service) {
+        if (serviceRules != null) {
+            UrlRewriteRulesDescriptor clusterRules = context.getDescriptor("rewrite");
+            clusterRules.addRules(serviceRules);
+        }
+    }
+
+    private void contributeResources(DeploymentContext context, Service service) {
+        Map<String, String> filterParams = new HashMap<String, String>();
+        List<UrlBinding> bindings = serviceDefinition.getUrlBindings();
+        for (UrlBinding binding : bindings) {
+            List<RewriteFilter> filters = binding.getRewriteFilters();
+            if (filters != null && !filters.isEmpty()) {
+                filterParams.clear();
+                for (RewriteFilter filter : filters) {
+                    filterParams.put(filter.getApplyTo(), filter.getRef());
+                }
+            }
+            try {
+                contributeResource(context, service, binding.getPattern(), filterParams);
+            } catch (URISyntaxException e) {
+                e.printStackTrace();
+            }
+        }
+
+    }
+
+    private void contributeResource(DeploymentContext context, Service service, String pattern, Map<String, String> filterParams) throws URISyntaxException {
+        List<FilterParamDescriptor> params = new ArrayList<FilterParamDescriptor>();
+        ResourceDescriptor resource = context.getGatewayDescriptor().addResource();
+        resource.role(service.getRole());
+        resource.pattern(pattern);
+        addWebAppSecFilters(context, service, resource);
+        addAuthenticationFilter(context, service, resource);
+        addIdentityAssertionFilter(context, service, resource);
+        addAuthorizationFilter(context, service, resource);
+        if (!filterParams.isEmpty()) {
+            for (Map.Entry<String, String> filterParam : filterParams.entrySet()) {
+                params.add(resource.createFilterParam().name(filterParam.getKey()).value(filterParam.getValue()));
+            }
+        }
+        addRewriteFilter( context, service, resource, params );
+        addDispatchFilter( context, service, resource, "dispatch", "http-client" );
+    }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-server/src/main/java/org/apache/hadoop/gateway/util/ServiceDefinitionsLoader.java
----------------------------------------------------------------------
diff --git a/gateway-server/src/main/java/org/apache/hadoop/gateway/util/ServiceDefinitionsLoader.java b/gateway-server/src/main/java/org/apache/hadoop/gateway/util/ServiceDefinitionsLoader.java
new file mode 100644
index 0000000..3d41605
--- /dev/null
+++ b/gateway-server/src/main/java/org/apache/hadoop/gateway/util/ServiceDefinitionsLoader.java
@@ -0,0 +1,95 @@
+/**
+ * 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.util;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.IOFileFilter;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.gateway.deploy.ServiceDeploymentContributor;
+import org.apache.hadoop.gateway.deploy.impl.ServiceDefinitionDeploymentContributor;
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteRulesDescriptor;
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteRulesDescriptorFactory;
+import org.apache.hadoop.gateway.service.definition.ServiceDefinition;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import java.io.*;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+public class ServiceDefinitionsLoader {
+
+    public static Set<ServiceDeploymentContributor> loadServiceDefinitions(File servicesDir) {
+        Set<ServiceDeploymentContributor> contributors = new HashSet<ServiceDeploymentContributor>();
+        if (servicesDir.exists() && servicesDir.isDirectory()) {
+            JAXBContext context = null;
+            try {
+                context = JAXBContext.newInstance(ServiceDefinition.class);
+                Unmarshaller unmarshaller = context.createUnmarshaller();
+                Collection<File> files = FileUtils.listFiles(servicesDir, new IOFileFilter() {
+                    @Override
+                    public boolean accept(File file) {
+                        return file.getName().contains("service");
+                    }
+
+                    @Override
+                    public boolean accept(File dir, String name) {
+                        return name.contains("service");
+                    }
+                }, TrueFileFilter.INSTANCE);
+                for (File file : files) {
+                    try {
+                        FileInputStream inputStream = new FileInputStream(file);
+                        ServiceDefinition definition = (ServiceDefinition) unmarshaller.unmarshal(inputStream);
+                        //look for rewrite rules as a sibling (for now)
+                        UrlRewriteRulesDescriptor rewriteRulesDescriptor = loadRewriteRules(file.getParentFile());
+                        contributors.add(new ServiceDefinitionDeploymentContributor(definition, rewriteRulesDescriptor));
+                    } catch (FileNotFoundException e) {
+                        e.printStackTrace();
+                    }
+                }
+            } catch (JAXBException e) {
+                e.printStackTrace();
+            }
+        }
+        return contributors;
+    }
+
+    private static UrlRewriteRulesDescriptor loadRewriteRules(File servicesDir) {
+        File rewriteFile = new File(servicesDir, "rewrite.xml");
+        if (rewriteFile.exists()) {
+            InputStream stream = null;
+            try {
+                stream = new FileInputStream(rewriteFile);
+                Reader reader = new InputStreamReader(stream);
+                UrlRewriteRulesDescriptor rules = UrlRewriteRulesDescriptorFactory.load(
+                        "xml", reader);
+                reader.close();
+                stream.close();
+                return rules;
+            } catch (FileNotFoundException e) {
+                e.printStackTrace();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/pom.xml
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/pom.xml b/gateway-service-definitions/pom.xml
new file mode 100644
index 0000000..004aed8
--- /dev/null
+++ b/gateway-service-definitions/pom.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0"?>
+<project
+        xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+        xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.knox</groupId>
+        <artifactId>gateway</artifactId>
+        <version>0.6.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>gateway-service-definitions</artifactId>
+    <name>gateway-service-definitions</name>
+    <description>The service definitions aka stacks.</description>
+    <licenses>
+        <license>
+            <name>The Apache Software License, Version 2.0</name>
+            <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+            <distribution>repo</distribution>
+        </license>
+    </licenses>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.eclipse.persistence</groupId>
+            <artifactId>eclipselink</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.knox</groupId>
+            <artifactId>gateway-test-utils</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/PolicyBinding.java
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/PolicyBinding.java b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/PolicyBinding.java
new file mode 100644
index 0000000..ad6b0a6
--- /dev/null
+++ b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/PolicyBinding.java
@@ -0,0 +1,21 @@
+/**
+ * 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.service.definition;
+
+public class PolicyBinding {
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/RewriteFilter.java
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/RewriteFilter.java b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/RewriteFilter.java
new file mode 100644
index 0000000..4b0dccc
--- /dev/null
+++ b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/RewriteFilter.java
@@ -0,0 +1,47 @@
+/**
+ * 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.service.definition;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlType;
+
+@XmlType(name = "rewrite-filter")
+public class RewriteFilter {
+
+  private String ref;
+
+  private String applyTo;
+
+  @XmlAttribute
+  public String getRef() {
+    return ref;
+  }
+
+  public void setRef(String ref) {
+    this.ref = ref;
+  }
+
+  @XmlAttribute(name = "apply-to")
+  public String getApplyTo() {
+    return applyTo;
+  }
+
+  public void setApplyTo(String applyTo) {
+    this.applyTo = applyTo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/ServiceDefinition.java
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/ServiceDefinition.java b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/ServiceDefinition.java
new file mode 100644
index 0000000..9bb5679
--- /dev/null
+++ b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/ServiceDefinition.java
@@ -0,0 +1,73 @@
+/**
+ * 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.service.definition;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.List;
+
+@XmlRootElement(name = "service")
+public class ServiceDefinition {
+
+  private String name;
+
+  private String role;
+
+  private String version;
+
+  private List<UrlBinding> urlBindings;
+
+  @XmlAttribute
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  @XmlAttribute
+  public String getRole() {
+    return role;
+  }
+
+  public void setRole(String role) {
+    this.role = role;
+  }
+
+  @XmlAttribute
+  public String getVersion() {
+    return version;
+  }
+
+  public void setVersion(String version) {
+    this.version = version;
+  }
+
+  @XmlElement(name = "url")
+  @XmlElementWrapper(name = "urls")
+  public List<UrlBinding> getUrlBindings() {
+    return urlBindings;
+  }
+
+  public void setUrlBindings(List<UrlBinding> urlBindings) {
+    this.urlBindings = urlBindings;
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/UrlBinding.java
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/UrlBinding.java b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/UrlBinding.java
new file mode 100644
index 0000000..3630ebf
--- /dev/null
+++ b/gateway-service-definitions/src/main/java/org/apache/hadoop/gateway/service/definition/UrlBinding.java
@@ -0,0 +1,49 @@
+/**
+ * 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.service.definition;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlType;
+import java.util.List;
+
+@XmlType(name = "url")
+public class UrlBinding {
+
+  private String pattern;
+
+  private List<RewriteFilter> rewriteFilters;
+
+  @XmlAttribute
+  public String getPattern() {
+    return pattern;
+  }
+
+  public void setPattern(String pattern) {
+    this.pattern = pattern;
+  }
+
+  @XmlElement(name = "rewrite-filter")
+  public List<RewriteFilter> getRewriteFilters() {
+    return rewriteFilters;
+  }
+
+  public void setRewriteFilters(List<RewriteFilter> rewriteFilters) {
+    this.rewriteFilters = rewriteFilters;
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/rewrite.xml
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/rewrite.xml b/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/rewrite.xml
new file mode 100644
index 0000000..95ce2a3
--- /dev/null
+++ b/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/rewrite.xml
@@ -0,0 +1,193 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+   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.
+-->
+<rules>
+
+    <rule dir="IN" name="RESOURCEMANAGER/resourcemanager/inbound/root" pattern="*://*:*/**/resourcemanager/v1/?{**}">
+        <rewrite template="{$serviceUrl[RESOURCEMANAGER]}/v1/?{**}"/>
+    </rule>
+
+    <rule dir="IN" name="RESOURCEMANAGER/resourcemanager/inbound/path" pattern="*://*:*/**/resourcemanager/v1/{path=**}?{**}">
+        <rewrite template="{$serviceUrl[RESOURCEMANAGER]}/v1/{path=**}/?{**}"/>
+    </rule>
+
+    <rule dir="IN" name="RESOURCEMANAGER/resourcemanager/inbound/proxy" pattern="*://*:*/**/resourcemanager/proxy/{appid=*}/ws/v1/{path=**}?{**}">
+        <decrypt-query/>
+        <match pattern="*://*:*/**/resourcemanager/proxy/{appid=*}/ws/v1/{path=**}?{scheme}?{host}?{port}?{**}"/>
+        <rewrite template="{scheme}://{host}:{port}/proxy/{appid=*}/ws/v1/{path=**}?{**}"/>
+    </rule>
+
+    <rule dir="OUT" name="RESOURCEMANAGER/resourcemanager/url/outbound">
+        <match pattern="*://*:*/**?**"/>
+		<rewrite template=""/>
+    </rule>
+    <rule dir="OUT" name="RESOURCEMANAGER/resourcemanager/trackingUrlHistory/outbound">
+        <match pattern="{scheme}://{host}:{port}/proxy/{appid=*}/jobhistory/job/**"/>
+		<rewrite template=""/>
+    </rule>
+    <rule dir="OUT" name="RESOURCEMANAGER/resourcemanager/trackingUrlAM/outbound">
+        <match pattern="{scheme}://{host}:{port}/proxy/{appid=*}"/>
+		<rewrite template="{$frontend[url]}/resourcemanager/proxy/{appid=*}?{scheme}?host={$hostmap(host)}?{port}?{**}"/>
+		<encrypt-query/>
+    </rule>
+    <rule dir="OUT" name="RESOURCEMANAGER/resourcemanager/hostport/outbound">
+        <match pattern="*:*"/>
+		<rewrite template=""/>
+    </rule>
+    <rule dir="OUT" name="RESOURCEMANAGER/resourcemanager/logsLink/outbound">
+        <match pattern="//*:*/**?**"/>
+		<rewrite template=""/>
+    </rule>
+    <rule dir="OUT" name="RESOURCEMANAGER/resourcemanager/host/outbound">
+		<rewrite template=""/>
+    </rule>
+    <rule dir="OUT" name="RESOURCEMANAGER/resourcemanager/nodeId/outbound">
+        <match pattern="{host=*}:{port=*}"/>
+        <encrypt template="{host}:{port}" param="encaddr"/>
+        <rewrite template="{encaddr}"/>
+    </rule>
+    <rule dir="IN" name="RESOURCEMANAGER/resourcemanager/nodeId/inbound">
+        <match pattern="*://*:*/**/resourcemanager/v1/cluster/nodes/{addr=*}?{**}"/>
+        <decrypt param="addr"/>
+        <rewrite template="{$serviceUrl[RESOURCEMANAGER]}/v1/cluster/nodes/{addr}?{**}"/>
+    </rule>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/apps/outbound">
+        <content type="*/json">
+        	<buffer path="$.apps.app[*]">
+        		<detect path="$.trackingUI" value="History">
+					<apply path="$.trackingUrl" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+				</detect>
+				<detect path="$.trackingUI" value="ApplicationMaster">
+					<apply path="$.trackingUrl" rule="RESOURCEMANAGER/resourcemanager/trackingUrlAM/outbound"/>
+				</detect>
+				<apply path="$.amContainerLogs" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+            	<apply path="$.amHostHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+        	</buffer>
+        </content>
+        <content type="*/xml">
+        	<buffer path="/apps/app">
+        		<detect path="trackingUI" value="History">
+					<apply path="trackingUrl" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+				</detect>
+				<detect path="trackingUI" value="ApplicationMaster">
+					<apply path="trackingUrl" rule="RESOURCEMANAGER/resourcemanager/trackingUrlAM/outbound"/>
+				</detect>
+				<apply path="amContainerLogs" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+            	<apply path="amHostHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+        	</buffer>
+        </content>
+    </filter>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/app/outbound">
+        <content type="*/json">
+        	<buffer path="$.app">
+	       		<detect path="$.trackingUI" value="History">
+	       			<apply path="$.trackingUrl" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+	       		</detect>
+	       		<detect path="$.trackingUI" value="ApplicationMaster">
+	       			<apply path="$.trackingUrl" rule="RESOURCEMANAGER/resourcemanager/trackingUrlAM/outbound"/>
+	       		</detect>
+	            <apply path="$.amContainerLogs" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+	            <apply path="$.amHostHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            </buffer>
+        </content>
+        <content type="*/xml">
+        	<buffer path="/app">
+	       		<detect path="trackingUI" value="History">
+	       			<apply path="trackingUrl" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+	       		</detect>
+	       		<detect path="trackingUI" value="ApplicationMaster">
+	       			<apply path="trackingUrl" rule="RESOURCEMANAGER/resourcemanager/trackingUrlAM/outbound"/>
+	       		</detect>
+	            <apply path="amContainerLogs" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+	            <apply path="amHostHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            </buffer>
+        </content>
+    </filter>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/appattempts/outbound">
+        <content type="*/json">
+            <apply path="$.appAttempts.appAttempt[*].nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="$.appAttempts.appAttempt[*].logsLink" rule="RESOURCEMANAGER/resourcemanager/logsLink/outbound"/>
+            <apply path="$.appAttempts.appAttempt[*].nodeId" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+        </content>
+        <content type="*/xml">
+            <apply path="/appAttempts/appAttempt/nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="/appAttempts/appAttempt/logsLink" rule="RESOURCEMANAGER/resourcemanager/logsLink/outbound"/>
+            <apply path="/appAttempts/appAttempt/nodeId" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+        </content>
+    </filter>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/nodes/outbound">
+        <content type="*/json">
+            <apply path="$.nodes.node[*].nodeHTTPAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="$.nodes.node[*].nodeHostName" rule="RESOURCEMANAGER/resourcemanager/host/outbound"/>
+            <apply path="$.nodes.node[*].id" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+        </content>
+        <content type="*/xml">
+            <apply path="/nodes/node/nodeHTTPAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="/nodes/node/nodeHostName" rule="RESOURCEMANAGER/resourcemanager/host/outbound"/>
+            <apply path="/nodes/node/id" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+        </content>
+    </filter>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/node/outbound">
+        <content type="*/json">
+            <apply path="$.node.nodeHTTPAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="$.node.nodeHostName" rule="RESOURCEMANAGER/resourcemanager/host/outbound"/>
+            <apply path="$.node.id" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+        </content>
+        <content type="*/xml">
+            <apply path="/node/nodeHTTPAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="/node/nodeHostName" rule="RESOURCEMANAGER/resourcemanager/host/outbound"/>
+            <apply path="/node/id" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+        </content>
+    </filter>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/proxy/jobattempts/outbound">
+        <content type="*/json">
+            <apply path="$.jobAttempts.jobAttempt[*].nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="$.jobAttempts.jobAttempt[*].nodeId" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+            <apply path="$.jobAttempts.jobAttempt[*].logsLink" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+        </content>
+        <content type="*/xml">
+            <apply path="/jobAttempts/jobAttempt/nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+            <apply path="/jobAttempts/jobAttempt/nodeId" rule="RESOURCEMANAGER/resourcemanager/nodeId/outbound"/>
+            <apply path="/jobAttempts/jobAttempt/logsLink" rule="RESOURCEMANAGER/resourcemanager/url/outbound"/>
+        </content>
+    </filter>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/proxy/taskattempts/outbound">
+        <content type="*/json">
+            <apply path="$.taskAttempts.taskAttempt[*].nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+        </content>
+        <content type="*/xml">
+            <apply path="/taskAttempts/taskAttempt/nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+        </content>
+    </filter>
+
+    <filter name="RESOURCEMANAGER/resourcemanager/proxy/taskattempt/outbound">
+        <content type="*/json">
+            <apply path="$.taskAttempt.nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+        </content>
+        <content type="*/xml">
+            <apply path="/taskAttempt/nodeHttpAddress" rule="RESOURCEMANAGER/resourcemanager/hostport/outbound"/>
+        </content>
+    </filter>
+
+</rules>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/service.xml
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/service.xml b/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/service.xml
new file mode 100644
index 0000000..8b53cca
--- /dev/null
+++ b/gateway-service-definitions/src/main/resources/services/yarn-rm/2.5.0/service.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+   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.
+-->
+<service role="RESOURCEMANAGER" name="resourcemanager" version="2.5.0">
+    <urls>
+        <url pattern="/resourcemanager/v1/cluster/"/>
+        <url pattern="/resourcemanager/v1/cluster/**?**"/>
+        <url pattern="/resourcemanager/v1/cluster/apps?**">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/apps/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/v1/cluster/apps?**">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/apps/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/v1/cluster/apps/*?**">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/app/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/v1/cluster/apps/*/appattempts?**">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/appattempts/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/v1/cluster/nodes?**">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/nodes/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/v1/cluster/nodes/*?**">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/nodeId/inbound" apply-to="request.url"/>
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/node/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/proxy/*/ws/v1/**?**">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/inbound/proxy" apply-to="request.url"/>
+        </url>
+        <url pattern="/resourcemanager/proxy/*/ws/v1/mapreduce/jobs/*/jobattempts">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/proxy/jobattempts/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/proxy/*/ws/v1/mapreduce/jobs/*/tasks/*/attempts">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/proxy/taskattempts/outbound" apply-to="response.body"/>
+        </url>
+        <url pattern="/resourcemanager/proxy/*/ws/v1/mapreduce/jobs/*/tasks/*/attempts/*">
+            <rewrite-filter ref="RESOURCEMANAGER/resourcemanager/proxy/taskattempt/outbound" apply-to="response.body"/>
+        </url>
+    </urls>
+</service>

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-definitions/src/test/java/org/apache/hadoop/gateway/service/definition/ServiceDefinitionTest.java
----------------------------------------------------------------------
diff --git a/gateway-service-definitions/src/test/java/org/apache/hadoop/gateway/service/definition/ServiceDefinitionTest.java b/gateway-service-definitions/src/test/java/org/apache/hadoop/gateway/service/definition/ServiceDefinitionTest.java
new file mode 100644
index 0000000..4f82af2
--- /dev/null
+++ b/gateway-service-definitions/src/test/java/org/apache/hadoop/gateway/service/definition/ServiceDefinitionTest.java
@@ -0,0 +1,46 @@
+/**
+ * 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.service.definition;
+
+import org.junit.Test;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Unmarshaller;
+import java.net.URL;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class ServiceDefinitionTest {
+
+  @Test
+  public void testUnmarshalling() throws Exception {
+    JAXBContext context = JAXBContext.newInstance(ServiceDefinition.class);
+    Unmarshaller unmarshaller = context.createUnmarshaller();
+    URL url = ClassLoader.getSystemResource("services/yarn-rm/2.5.0/service.xml");
+    ServiceDefinition definition = (ServiceDefinition) unmarshaller.unmarshal(url.openStream());
+    assertEquals("resourcemanager", definition.getName());
+    assertEquals("RESOURCEMANAGER", definition.getRole());
+    assertEquals("2.5.0", definition.getVersion());
+    List<UrlBinding> bindings = definition.getUrlBindings();
+    assertNotNull(bindings);
+    assertEquals(12, bindings.size());
+    assertNotNull(bindings.get(0).getPattern());
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-service-yarn-rm/src/main/java/org/apache/hadoop/gateway/yarn/rm/ResourceManagerDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-yarn-rm/src/main/java/org/apache/hadoop/gateway/yarn/rm/ResourceManagerDeploymentContributor.java b/gateway-service-yarn-rm/src/main/java/org/apache/hadoop/gateway/yarn/rm/ResourceManagerDeploymentContributor.java
index 21e1b2e..43746d1 100644
--- a/gateway-service-yarn-rm/src/main/java/org/apache/hadoop/gateway/yarn/rm/ResourceManagerDeploymentContributor.java
+++ b/gateway-service-yarn-rm/src/main/java/org/apache/hadoop/gateway/yarn/rm/ResourceManagerDeploymentContributor.java
@@ -46,19 +46,20 @@ public class ResourceManagerDeploymentContributor extends
 
   @Override
   public String getRole() {
-    return "RESOURCEMANAGER";
+    return "xRESOURCEMANAGER";
   }
 
   @Override
   public String getName() {
-    return "resourcemanager";
+    return "xresourcemanager";
   }
 
   @Override
   public void contributeService(DeploymentContext context, Service service)
       throws Exception {
-    contributeRewriteRules( context, service );
-    contributeResources( context, service );
+      //TODO: [sumit] delete when done with Stacks work KNOX-481
+//    contributeRewriteRules( context, service );
+//    contributeResources( context, service );
   }
 
   private void contributeRewriteRules(DeploymentContext context, Service service)

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-spi/src/main/java/org/apache/hadoop/gateway/config/GatewayConfig.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/config/GatewayConfig.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/config/GatewayConfig.java
index 010e25d..7d20e1a 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/config/GatewayConfig.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/config/GatewayConfig.java
@@ -52,6 +52,12 @@ public interface GatewayConfig {
    */
   String getGatewayDataDir();
 
+  /**
+   * The location of the gateway stack definitions
+   * @return The location of the gateway stacks top level directory.
+   */
+  String getGatewayStacksDir();
+
   String getHadoopConfDir();
 
   String getGatewayHost();

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayFuncTestDriver.java
----------------------------------------------------------------------
diff --git a/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayFuncTestDriver.java b/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayFuncTestDriver.java
index 6faccac..6117c76 100644
--- a/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayFuncTestDriver.java
+++ b/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayFuncTestDriver.java
@@ -157,6 +157,18 @@ public class GatewayFuncTestDriver {
     } catch (ServiceLifecycleException e) {
       e.printStackTrace(); // I18N not required.
     }
+    File stacksDir = new File( config.getGatewayStacksDir() );
+    stacksDir.mkdirs();
+    //TODO: [sumit] This is a hack for now, need to find a better way to locate the source resources for 'stacks' to be tested
+    String pathToStacksSource = "gateway-service-definitions/src/main/resources/services";
+    File stacksSourceDir = new File( targetDir.getParent(), pathToStacksSource);
+    if (!stacksSourceDir.exists()) {
+      stacksSourceDir = new File( targetDir.getParentFile().getParent(), pathToStacksSource);
+    }
+    if (stacksSourceDir.exists()) {
+      FileUtils.copyDirectoryToDirectory(stacksSourceDir, stacksDir);
+    }
+
     gateway = GatewayServer.startGateway( config, srvcs );
     MatcherAssert.assertThat( "Failed to start gateway.", gateway, notNullValue() );
 
@@ -170,6 +182,7 @@ public class GatewayFuncTestDriver {
     FileUtils.deleteQuietly( new File( config.getGatewaySecurityDir() ) );
     FileUtils.deleteQuietly( new File( config.getGatewayDeploymentDir() ) );
     FileUtils.deleteQuietly( new File( config.getGatewayDataDir() ) );
+    FileUtils.deleteQuietly( new File( config.getGatewayStacksDir() ) );
 
     for( Service service : services.values() ) {
       service.server.stop();

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayTestConfig.java
----------------------------------------------------------------------
diff --git a/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayTestConfig.java b/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayTestConfig.java
index a013505..ec333fe 100644
--- a/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayTestConfig.java
+++ b/gateway-test/src/test/java/org/apache/hadoop/gateway/GatewayTestConfig.java
@@ -189,5 +189,9 @@ public class GatewayTestConfig implements GatewayConfig {
 //  public void setKerberosLoginConfig(String kerberosLoginConfig) {
 //   this.kerberosLoginConfig = kerberosLoginConfig;
 //  }
-  
+
+   @Override
+   public String getGatewayStacksDir() {
+      return gatewayHomeDir + "/data/stacks";
+   }
 }

http://git-wip-us.apache.org/repos/asf/knox/blob/6a209042/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e121942..c2112fe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -69,6 +69,7 @@
         <module>gateway-service-webhcat</module>
         <module>gateway-service-tgs</module>
         <module>gateway-service-yarn-rm</module>
+        <module>gateway-service-definitions</module>
         <module>gateway-shell</module>
         <module>gateway-shell-launcher</module>
         <module>knox-cli-launcher</module>
@@ -496,6 +497,11 @@
             </dependency>
             <dependency>
                 <groupId>${gateway-group}</groupId>
+                <artifactId>gateway-service-definitions</artifactId>
+                <version>${gateway-version}</version>
+            </dependency>
+            <dependency>
+                <groupId>${gateway-group}</groupId>
                 <artifactId>gateway-service-oozie</artifactId>
                 <version>${gateway-version}</version>
             </dependency>