You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@knox.apache.org by km...@apache.org on 2013/09/20 05:58:52 UTC

[3/3] git commit: KNOX-127: Use hostmap in some service registry rewrite functions.

KNOX-127: Use hostmap in some service registry rewrite functions.


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

Branch: refs/heads/master
Commit: e338fe64f94dfa5365b699faa6f8621ce3f59bf2
Parents: afe298e
Author: Kevin Minder <ke...@hortonworks.com>
Authored: Thu Sep 19 23:58:05 2013 -0400
Committer: Kevin Minder <ke...@hortonworks.com>
Committed: Thu Sep 19 23:58:05 2013 -0400

----------------------------------------------------------------------
 .../function/UsernameFunctionProcessor.java     |  18 +-
 .../function/UsernameFunctionProcessorTest.java |   9 +-
 .../impl/HostmapDeploymentContributor.java      |  13 +-
 .../hostmap/impl/HostmapFunctionProcessor.java  |  73 ++++----
 .../impl/HostmapFunctionProcessorTest.java      |  36 +++-
 .../api/ServiceAddressFunctionDescriptor.java   |  11 +-
 .../api/ServiceHostFunctionDescriptor.java      |  11 +-
 .../ServiceMappedAddressFunctionDescriptor.java |  31 ++++
 .../ServiceMappedHostFunctionDescriptor.java    |  31 ++++
 .../api/ServiceMappedUrlFunctionDescriptor.java |  31 ++++
 .../api/ServicePathFunctionDescriptor.java      |  11 +-
 .../api/ServicePortFunctionDescriptor.java      |  11 +-
 .../api/ServiceSchemeFunctionDescriptor.java    |  11 +-
 .../impl/ServiceAddressFunctionProcessor.java   |  40 ++++-
 .../impl/ServiceHostFunctionProcessor.java      |  28 +++-
 .../ServiceMappedAddressFunctionProcessor.java  | 110 ++++++++++++
 .../ServiceMappedHostFunctionProcessor.java     |  87 ++++++++++
 .../impl/ServiceMappedUrlFunctionProcessor.java |  90 ++++++++++
 .../impl/ServicePathFunctionProcessor.java      |  28 ++--
 .../impl/ServicePortFunctionProcessor.java      |  28 +++-
 .../ServiceRegistryFunctionProcessorBase.java   |  19 +--
 .../impl/ServiceSchemeFunctionProcessor.java    |  28 +++-
 .../impl/ServiceUrlFunctionProcessor.java       |  31 +++-
 ...ter.rewrite.api.UrlRewriteFunctionDescriptor |   5 +-
 ...lter.rewrite.spi.UrlRewriteFunctionProcessor |   5 +-
 .../ServiceAddressFunctionProcessorTest.java    |   7 +-
 .../impl/ServiceHostFunctionProcessorTest.java  |   8 +-
 ...rviceMappedAddressFunctionProcessorTest.java | 148 ++++++++++++++++
 .../ServiceMappedHostFunctionProcessorTest.java | 148 ++++++++++++++++
 .../ServiceMappedUrlFunctionProcessorTest.java  | 148 ++++++++++++++++
 .../impl/ServicePathFunctionProcessorTest.java  |   6 +-
 .../impl/ServicePortFunctionProcessorTest.java  |   8 +-
 .../ServiceSchemeFunctionProcessorTest.java     |   7 +-
 .../impl/ServiceUrlFunctionProcessorTest.java   |  25 ++-
 .../filter/rewrite/api/UrlRewriteProcessor.java |   9 +-
 .../UrlRewriteActionRewriteProcessorExt.java    |   2 +-
 .../rewrite/impl/UrlRewriteContextImpl.java     |  47 ++++--
 .../impl/UrlRewriteFunctionResolver.java        | 115 -------------
 .../filter/rewrite/spi/UrlRewriteContext.java   |   3 +
 .../filter/rewrite/spi/UrlRewriteResolver.java  |   4 +-
 .../rewrite/impl/UrlRewriteContextImplTest.java |  14 +-
 gateway-release/home/deployments/sample.xml     |   4 +-
 .../apache/hadoop/gateway/GatewayMessages.java  |   9 -
 .../gateway/dispatch/UrlConnectionDispatch.java |   2 +-
 .../services/DefaultGatewayServices.java        |   4 +-
 .../gateway/services/HssoGatewayServices.java   |  17 +-
 .../hostmap/impl/DefaultHostMapperService.java  |  81 +++++++++
 .../hostmap/impl/DefaultHostMappingService.java |  83 ---------
 .../WebHdfsDeploymentContributor/rewrite.xml    |   2 +-
 .../hadoop/gateway/i18n/GatewaySpiMessages.java |   9 +
 .../services/hostmap/FileBasedHostMapper.java   |  61 +++----
 .../gateway/services/hostmap/HostMapper.java    |  12 +-
 .../services/hostmap/HostMapperService.java     |  41 +++++
 .../services/hostmap/HostMappingService.java    |  35 ----
 .../security/impl/CMFMasterService.java         |  17 +-
 .../hostmap/FileBasedHostMapperTest.java        |  64 +++++++
 .../hostmap/FileBasedHostMapperTest/hostmap.txt |   6 +
 .../gateway/util/urltemplate/Evaluator.java     |  26 +++
 .../gateway/util/urltemplate/Expander.java      | 110 ++++++------
 .../gateway/util/urltemplate/Function.java      | 114 +++++++++++++
 .../gateway/util/urltemplate/MockParams.java    |   1 +
 .../gateway/util/urltemplate/Rewriter.java      |   9 +-
 .../gateway/util/urltemplate/ExpanderTest.java  |  68 ++++----
 .../gateway/util/urltemplate/FunctionTest.java  | 167 +++++++++++++++++++
 .../gateway/util/urltemplate/MatcherTest.java   |  99 +++++++++++
 .../gateway/util/urltemplate/ParserTest.java    |   8 +-
 .../gateway/util/urltemplate/RewriterTest.java  |  66 +++++---
 67 files changed, 1991 insertions(+), 619 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessor.java b/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessor.java
index 8cabeab..b3c90f5 100644
--- a/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessor.java
+++ b/gateway-provider-identity-assertion-pseudo/src/main/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessor.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.gateway.i18n.messages.MessagesFactory;
 
 import javax.security.auth.Subject;
 import java.security.AccessController;
+import java.util.ArrayList;
+import java.util.List;
 
 public class UsernameFunctionProcessor
     extends AbstractIdentityAssertionBase
@@ -54,16 +56,18 @@ public class UsernameFunctionProcessor
   }
 
   @Override
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    String value = parameter;
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
     Subject subject = Subject.getSubject( AccessController.getContext() );
     if( subject != null ) {
-      value = getPrincipalName( subject );
-//      if( mapper != null ) {
-//        value = mapper.mapPrincipal( value );
-//      }
+      results = new ArrayList<String>( 1 );
+      String username = getPrincipalName( subject );
+      results.add( username );
+    } else if( parameters != null && parameters.size() > 0 ) {
+      results = new ArrayList<String>( 1 );
+      results.add( parameters.get( 0 ) );
     }
-    return value;
+    return results;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessorTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessorTest.java b/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessorTest.java
index dab9c59..dffa4b0 100644
--- a/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessorTest.java
+++ b/gateway-provider-identity-assertion-pseudo/src/test/java/org/apache/hadoop/gateway/identityasserter/function/UsernameFunctionProcessorTest.java
@@ -52,15 +52,16 @@ import java.net.URL;
 import java.nio.charset.Charset;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.ServiceLoader;
 
-import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.nullValue;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
 import static org.junit.Assert.fail;
 
 public class UsernameFunctionProcessorTest {
@@ -135,15 +136,15 @@ public class UsernameFunctionProcessorTest {
   public void testResolve() throws Exception {
     final UsernameFunctionProcessor processor = new UsernameFunctionProcessor();
     assertThat( processor.resolve( null, null ), nullValue() );
-    assertThat( processor.resolve( null, "test-input" ), is( "test-input" ) );
+    assertThat( processor.resolve( null, Arrays.asList( "test-input" ) ), contains( "test-input" ) );
     Subject subject = new Subject();
     subject.getPrincipals().add( new PrimaryPrincipal( "test-username" ) );
     subject.setReadOnly();
     Subject.doAs( subject, new PrivilegedExceptionAction<Object>() {
       @Override
       public Object run() throws Exception {
-        assertThat( processor.resolve( null, null ), is( "test-username" ) );
-        assertThat( processor.resolve( null, "test-ignored" ), is( "test-username" ) );
+        assertThat( processor.resolve( null, null ), contains( "test-username" ) );
+        assertThat( processor.resolve( null, Arrays.asList( "test-ignored" ) ), contains( "test-username" ) );
         return null;
       }
     } );

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapDeploymentContributor.java b/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapDeploymentContributor.java
index f79b046..fdd0b03 100644
--- a/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapDeploymentContributor.java
+++ b/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapDeploymentContributor.java
@@ -24,7 +24,6 @@ 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.hostmap.api.HostmapFunctionDescriptor;
-import org.apache.hadoop.gateway.services.hostmap.HostMappingService;
 import org.apache.hadoop.gateway.topology.Provider;
 import org.apache.hadoop.gateway.topology.Service;
 import org.jboss.shrinkwrap.api.asset.Asset;
@@ -41,14 +40,8 @@ public class HostmapDeploymentContributor
 
   public static final String PROVIDER_ROLE_NAME = HostmapFunctionDescriptor.FUNCTION_NAME;
   public static final String PROVIDER_IMPL_NAME = "static";
-
   private static final String REWRITE_ROLE_NAME = "rewrite";
-  private HostMappingService hostMapping;
-  
-  private void setHostMappingService(HostMappingService hostMapping) {
-    this.hostMapping = hostMapping;
-  }
-  
+
   @Override
   public String getRole() {
     return PROVIDER_ROLE_NAME;
@@ -80,12 +73,12 @@ public class HostmapDeploymentContributor
   private Asset createAsset( Provider provider ) {
     StringWriter buffer = new StringWriter();
     PrintWriter writer = new PrintWriter( buffer );
-    for( Map.Entry<String,String> entry : provider.getParams().entrySet() ) {
+    for( Map.Entry<String, String> entry : provider.getParams().entrySet() ) {
       String externalHosts = entry.getKey();
       String internalHosts = entry.getValue();
       writer.print( externalHosts );
       writer.print( "=" );
-      writer.println( internalHosts ) ;
+      writer.println( internalHosts );
     }
     writer.close();
     String string = buffer.toString();

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessor.java b/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessor.java
index 9be1dd2..ca55c16 100644
--- a/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessor.java
+++ b/gateway-provider-rewrite-func-hostmap-static/src/main/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessor.java
@@ -23,26 +23,22 @@ import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
 import org.apache.hadoop.gateway.hostmap.api.HostmapFunctionDescriptor;
 import org.apache.hadoop.gateway.services.GatewayServices;
 import org.apache.hadoop.gateway.services.hostmap.FileBasedHostMapper;
-import org.apache.hadoop.gateway.services.hostmap.HostMappingService;
-import org.apache.hadoop.gateway.services.security.CryptoService;
+import org.apache.hadoop.gateway.services.hostmap.HostMapper;
+import org.apache.hadoop.gateway.services.hostmap.HostMapperService;
 
-import java.io.BufferedReader;
-import java.io.InputStream;
-import java.io.InputStreamReader;
 import java.net.URL;
-import java.util.HashMap;
+import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 public class HostmapFunctionProcessor
     implements UrlRewriteFunctionProcessor<HostmapFunctionDescriptor> {
 
   public static final String DESCRIPTOR_DEFAULT_FILE_NAME = "hostmap.txt";
   public static final String DESCRIPTOR_DEFAULT_LOCATION = "/WEB-INF/" + DESCRIPTOR_DEFAULT_FILE_NAME;
-  
-  private FileBasedHostMapper hostMapper = null;
+
+  private HostMapperService hostMapperService;
+  private HostMapper hostMapper = null;
   private String clusterName;
-  private HostMappingService hostMappingService;
 
   @Override
   public String name() {
@@ -52,48 +48,43 @@ public class HostmapFunctionProcessor
   @Override
   public void initialize( UrlRewriteEnvironment environment, HostmapFunctionDescriptor descriptor ) throws Exception {
     URL url = environment.getResource( DESCRIPTOR_DEFAULT_LOCATION );
-    List<String> names = environment.resolve( "cluster.name" );
-    if (names != null && names.size() > 0) {
-      clusterName = names.get( 0 );
-    }
-    hostMapper = new FileBasedHostMapper(clusterName, url);
-
-    GatewayServices services = environment.getAttribute(GatewayServices.GATEWAY_SERVICES_ATTRIBUTE);
-    if (clusterName != null && services != null) {
-      hostMappingService = (HostMappingService) services.getService(GatewayServices.HOST_MAPPING_SERVICE);
-      if (hostMappingService != null) {
-        hostMappingService.registerHostMapperForCluster(clusterName, hostMapper);
+    hostMapper = new FileBasedHostMapper( url );
+    clusterName = environment.getAttribute(  GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE );
+    GatewayServices services = environment.getAttribute( GatewayServices.GATEWAY_SERVICES_ATTRIBUTE );
+    if( clusterName != null && services != null ) {
+      hostMapperService = services.getService( GatewayServices.HOST_MAPPING_SERVICE );
+      if( hostMapperService != null ) {
+        hostMapperService.registerHostMapperForCluster( clusterName, hostMapper );
       }
     }
   }
 
   @Override
   public void destroy() throws Exception {
-    // need to remove the host mapper for the cluster on undeploy
-    if (clusterName != null && hostMappingService != null) {
-      hostMappingService.removeHostMapperForCluster(clusterName);
+    if( hostMapperService != null && clusterName != null ) {
+      hostMapperService.removeHostMapperForCluster( clusterName );
     }
-    
   }
 
   @Override
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    String value;
-    switch( context.getDirection() ) {
-      case IN:
-        value = hostMapper.resolveInboundHostName(parameter);
-        break;
-      case OUT:
-        value = hostMapper.resolveOutboundHostName(parameter);
-        break;
-      default:
-        value = null;
-    }
-    if( value == null ) {
-      value = parameter;
-    }
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> result = null;
+    if( parameters != null ) {
+      result = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        switch( context.getDirection() ) {
+          case IN:
+            parameter = hostMapper.resolveInboundHostName( parameter );
+            break;
+          case OUT:
+            parameter = hostMapper.resolveOutboundHostName( parameter );
+            break;
+        }
+        result.add( parameter );
+      }
 //    System.out.println( "HOSTMAP: " + parameter + "->" + value );
-    return value;
+    }
+    return result;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-hostmap-static/src/test/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessorTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-hostmap-static/src/test/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessorTest.java b/gateway-provider-rewrite-func-hostmap-static/src/test/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessorTest.java
index 011a4bd..be738e9 100644
--- a/gateway-provider-rewrite-func-hostmap-static/src/test/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessorTest.java
+++ b/gateway-provider-rewrite-func-hostmap-static/src/test/java/org/apache/hadoop/gateway/hostmap/impl/HostmapFunctionProcessorTest.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.gateway.filter.rewrite.ext.UrlRewriteActionRewriteDescr
 import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
 import org.apache.hadoop.gateway.services.GatewayServices;
 import org.apache.hadoop.gateway.services.hostmap.HostMapper;
-import org.apache.hadoop.gateway.services.hostmap.HostMappingService;
+import org.apache.hadoop.gateway.services.hostmap.HostMapperService;
 import org.apache.hadoop.gateway.util.urltemplate.Parser;
 import org.apache.hadoop.gateway.util.urltemplate.Resolver;
 import org.apache.hadoop.gateway.util.urltemplate.Template;
@@ -68,7 +68,7 @@ public class HostmapFunctionProcessorTest {
     HostMapper hm = EasyMock.createNiceMock(HostMapper.class);
     EasyMock.expect( hm.resolveInboundHostName("test-inbound-host")).andReturn( "test-inbound-rewritten-host" ).anyTimes();
     
-    HostMappingService hms = EasyMock.createNiceMock( HostMappingService.class );
+    HostMapperService hms = EasyMock.createNiceMock( HostMapperService.class );
 
     GatewayServices gatewayServices = EasyMock.createNiceMock( GatewayServices.class );
     EasyMock.expect( gatewayServices.getService( GatewayServices.HOST_MAPPING_SERVICE ) ).andReturn( hms ).anyTimes();
@@ -194,7 +194,7 @@ public class HostmapFunctionProcessorTest {
   }
 
   @Test
-  public void testMissingFunctionUseCase() throws Exception {
+  public void testInvalidFunctionNameUseCase() throws Exception {
     URL configUrl = TestUtils.getResourceUrl( this.getClass(), "hostmap.txt" );
 
     UrlRewriteEnvironment environment = EasyMock.createNiceMock( UrlRewriteEnvironment.class );
@@ -216,11 +216,11 @@ public class HostmapFunctionProcessorTest {
     Template output = rewriter.rewrite( resolver, input, UrlRewriter.Direction.IN, null );
     //System.out.println( output );
     assertThat( output, notNullValue() );
-    assertThat( output.getHost().getFirstValue().getPattern(), is( "test-inbound-host" ) );
+    assertThat( output.getHost().getFirstValue().getPattern(), is( "$invalid-function(host)" ) );
   }
 
   @Test
-  public void testEmptyHostmapUseCase() throws Exception {
+  public void testInvalidFunctionNameAndEmptyHostmapUseCase() throws Exception {
     URL configUrl = TestUtils.getResourceUrl( this.getClass(), "empty-hostmap.txt" );
 
     UrlRewriteEnvironment environment = EasyMock.createNiceMock( UrlRewriteEnvironment.class );
@@ -242,6 +242,32 @@ public class HostmapFunctionProcessorTest {
     Template output = rewriter.rewrite( resolver, input, UrlRewriter.Direction.IN, null );
     //System.out.println( output );
     assertThat( output, notNullValue() );
+    assertThat( output.getHost().getFirstValue().getPattern(), is( "$invalid-function(host)" ) );
+  }
+
+  @Test
+  public void testEmptyHostmapUseCase() throws Exception {
+    URL configUrl = TestUtils.getResourceUrl( this.getClass(), "empty-hostmap.txt" );
+
+    UrlRewriteEnvironment environment = EasyMock.createNiceMock( UrlRewriteEnvironment.class );
+    EasyMock.expect( environment.getResource( "/WEB-INF/hostmap.txt" ) ).andReturn( configUrl ).anyTimes();
+    Resolver resolver = EasyMock.createNiceMock( Resolver.class );
+    EasyMock.expect( resolver.resolve( "host" ) ).andReturn( Arrays.asList( "test-inbound-host" ) ).anyTimes();
+    EasyMock.replay( environment, resolver );
+
+    UrlRewriteRulesDescriptor descriptor = UrlRewriteRulesDescriptorFactory.create();
+    UrlRewriteRuleDescriptor rule = descriptor.addRule( "test-rule" );
+    rule.pattern( "{*}://{host}:{*}/{**}?{**}" );
+    UrlRewriteActionRewriteDescriptorExt rewrite = rule.addStep( "rewrite" );
+    rewrite.template( "{*}://{$hostmap(host)}:{*}/{**}?{**}" );
+
+    UrlRewriteProcessor rewriter = new UrlRewriteProcessor();
+    rewriter.initialize( environment, descriptor );
+
+    Template input = Parser.parse( "test-scheme://test-inbound-host:42/test-path/test-file?test-name=test-value" );
+    Template output = rewriter.rewrite( resolver, input, UrlRewriter.Direction.IN, null );
+    //System.out.println( output );
+    assertThat( output, notNullValue() );
     assertThat( output.getHost().getFirstValue().getPattern(), is( "test-inbound-host" ) );
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceAddressFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceAddressFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceAddressFunctionDescriptor.java
index b271215..bfe8933 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceAddressFunctionDescriptor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceAddressFunctionDescriptor.java
@@ -21,10 +21,11 @@ import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
 
 public class ServiceAddressFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceUrlFunctionDescriptor> {
 
-    public static final String FUNCTION_NAME = "serviceAddr";
-      @Override
-      public String name() {
-      return FUNCTION_NAME;
-    }
+  public static final String FUNCTION_NAME = "serviceAddr";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceHostFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceHostFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceHostFunctionDescriptor.java
index 83a66c2..cff251b 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceHostFunctionDescriptor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceHostFunctionDescriptor.java
@@ -21,10 +21,11 @@ import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
 
 public class ServiceHostFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceUrlFunctionDescriptor> {
 
-    public static final String FUNCTION_NAME = "serviceHost";
-      @Override
-      public String name() {
-      return FUNCTION_NAME;
-    }
+  public static final String FUNCTION_NAME = "serviceHost";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedAddressFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedAddressFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedAddressFunctionDescriptor.java
new file mode 100644
index 0000000..ae12de0
--- /dev/null
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedAddressFunctionDescriptor.java
@@ -0,0 +1,31 @@
+/**
+ * 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.svcregfunc.api;
+
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor;
+
+public class ServiceMappedAddressFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceUrlFunctionDescriptor> {
+
+  public static final String FUNCTION_NAME = "serviceMappedAddr";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedHostFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedHostFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedHostFunctionDescriptor.java
new file mode 100644
index 0000000..dd8beb2
--- /dev/null
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedHostFunctionDescriptor.java
@@ -0,0 +1,31 @@
+/**
+ * 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.svcregfunc.api;
+
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor;
+
+public class ServiceMappedHostFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceUrlFunctionDescriptor> {
+
+  public static final String FUNCTION_NAME = "serviceMappedHost";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedUrlFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedUrlFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedUrlFunctionDescriptor.java
new file mode 100644
index 0000000..8ea8271
--- /dev/null
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceMappedUrlFunctionDescriptor.java
@@ -0,0 +1,31 @@
+/**
+ * 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.svcregfunc.api;
+
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor;
+
+public class ServiceMappedUrlFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceMappedUrlFunctionDescriptor> {
+
+  public static final String FUNCTION_NAME = "serviceMappedUrl";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePathFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePathFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePathFunctionDescriptor.java
index d819d61..ab69264 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePathFunctionDescriptor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePathFunctionDescriptor.java
@@ -21,10 +21,11 @@ import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
 
 public class ServicePathFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceUrlFunctionDescriptor> {
 
-    public static final String FUNCTION_NAME = "servicePath";
-      @Override
-      public String name() {
-      return FUNCTION_NAME;
-    }
+  public static final String FUNCTION_NAME = "servicePath";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePortFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePortFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePortFunctionDescriptor.java
index 8a04ad2..cc05975 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePortFunctionDescriptor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServicePortFunctionDescriptor.java
@@ -21,10 +21,11 @@ import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
 
 public class ServicePortFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceUrlFunctionDescriptor> {
 
-    public static final String FUNCTION_NAME = "servicePort";
-      @Override
-      public String name() {
-      return FUNCTION_NAME;
-    }
+  public static final String FUNCTION_NAME = "servicePort";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceSchemeFunctionDescriptor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceSchemeFunctionDescriptor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceSchemeFunctionDescriptor.java
index a391b72..dc73dee 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceSchemeFunctionDescriptor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/api/ServiceSchemeFunctionDescriptor.java
@@ -21,10 +21,11 @@ import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
 
 public class ServiceSchemeFunctionDescriptor implements UrlRewriteFunctionDescriptor<ServiceUrlFunctionDescriptor> {
 
-    public static final String FUNCTION_NAME = "serviceScheme";
-      @Override
-      public String name() {
-      return FUNCTION_NAME;
-    }
+  public static final String FUNCTION_NAME = "serviceScheme";
+
+  @Override
+  public String name() {
+    return FUNCTION_NAME;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessor.java
index 350b6f9..c8b53a3 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessor.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.gateway.svcregfunc.impl;
 
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriter;
 import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
 import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
 import org.apache.hadoop.gateway.svcregfunc.api.ServiceAddressFunctionDescriptor;
@@ -25,6 +26,9 @@ import org.apache.hadoop.gateway.util.urltemplate.Parser;
 import org.apache.hadoop.gateway.util.urltemplate.Port;
 import org.apache.hadoop.gateway.util.urltemplate.Template;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class ServiceAddressFunctionProcessor
     extends ServiceRegistryFunctionProcessorBase<ServiceAddressFunctionDescriptor>
     implements UrlRewriteFunctionProcessor<ServiceAddressFunctionDescriptor> {
@@ -34,19 +38,43 @@ public class ServiceAddressFunctionProcessor
     return ServiceAddressFunctionDescriptor.FUNCTION_NAME;
   }
 
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      UrlRewriter.Direction direction = context.getDirection();
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        parameter = resolve( parameter );
+        results.add( parameter );
+      }
+    }
+    return results;
+  }
+
+  public String resolve( String parameter ) throws Exception {
     String addr = parameter;
-    String url = super.resolve( context, parameter );
+    String url = lookupServiceUrl( parameter );
     if( url != null ) {
       Template template = Parser.parse( url );
       Host host = template.getHost();
+      String hostStr = null;
+      if( host != null ) {
+        hostStr = host.getFirstValue().getPattern();
+      }
+
       Port port = template.getPort();
-      if( host != null && port != null ) {
-        addr = host.getFirstValue().getPattern() + ":" + port.getFirstValue().getPattern();
+      String portStr = null;
+      if( port != null ) {
+        portStr = port.getFirstValue().getPattern();
+      }
+
+      if( hostStr != null && portStr != null ) {
+        addr = hostStr + ":" + portStr;
       } else if( host != null && port == null ) {
-        addr = host.getFirstValue().getPattern();
+        addr = hostStr;
       } else if( host == null && port != null ) {
-        addr = ":" + port.getFirstValue().getPattern();
+        addr = ":" + portStr;
       }
     }
     return addr;

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessor.java
index 2f50a82..c9e7f9b 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessor.java
@@ -24,6 +24,9 @@ import org.apache.hadoop.gateway.util.urltemplate.Host;
 import org.apache.hadoop.gateway.util.urltemplate.Parser;
 import org.apache.hadoop.gateway.util.urltemplate.Template;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class ServiceHostFunctionProcessor
     extends ServiceRegistryFunctionProcessorBase<ServiceHostFunctionDescriptor>
     implements UrlRewriteFunctionProcessor<ServiceHostFunctionDescriptor> {
@@ -33,17 +36,24 @@ public class ServiceHostFunctionProcessor
     return ServiceHostFunctionDescriptor.FUNCTION_NAME;
   }
 
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    String value = parameter;
-    String url = super.resolve( context, parameter );
-    if( url != null && !url.equals( parameter ) ) {
-      Template template = Parser.parse( url );
-      Host host = template.getHost();
-      if( host != null ) {
-        value = host.getFirstValue().getPattern();
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        String url = lookupServiceUrl( parameter );
+        if( url != null ) {
+          Template template = Parser.parse( url );
+          Host host = template.getHost();
+          if( host != null ) {
+            parameter = host.getFirstValue().getPattern();
+          }
+        }
+        results.add( parameter );
       }
     }
-    return value;
+    return results;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessor.java
new file mode 100644
index 0000000..5809eb8
--- /dev/null
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessor.java
@@ -0,0 +1,110 @@
+/**
+ * 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.svcregfunc.impl;
+
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteEnvironment;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
+import org.apache.hadoop.gateway.services.GatewayServices;
+import org.apache.hadoop.gateway.services.hostmap.HostMapper;
+import org.apache.hadoop.gateway.services.hostmap.HostMapperService;
+import org.apache.hadoop.gateway.svcregfunc.api.ServiceMappedAddressFunctionDescriptor;
+import org.apache.hadoop.gateway.util.urltemplate.Host;
+import org.apache.hadoop.gateway.util.urltemplate.Parser;
+import org.apache.hadoop.gateway.util.urltemplate.Port;
+import org.apache.hadoop.gateway.util.urltemplate.Template;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriter.Direction;
+
+public class ServiceMappedAddressFunctionProcessor
+    extends ServiceRegistryFunctionProcessorBase<ServiceMappedAddressFunctionDescriptor>
+    implements UrlRewriteFunctionProcessor<ServiceMappedAddressFunctionDescriptor> {
+
+  private HostMapper hostmap = null;
+
+  @Override
+  public String name() {
+    return ServiceMappedAddressFunctionDescriptor.FUNCTION_NAME;
+  }
+
+  @Override
+  public void initialize( UrlRewriteEnvironment environment, ServiceMappedAddressFunctionDescriptor descriptor ) throws Exception {
+    super.initialize( environment, descriptor );
+    HostMapperService hostmapService = services().getService( GatewayServices.HOST_MAPPING_SERVICE );
+    if( hostmapService != null ) {
+      hostmap = hostmapService.getHostMapper( cluster() );
+    }
+  }
+
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      Direction direction = context.getDirection();
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        parameter = resolve( direction, parameter );
+        results.add( parameter );
+      }
+    }
+    return results;
+  }
+
+  public String resolve( Direction direction, String parameter ) throws Exception {
+    String addr = parameter;
+    String url = lookupServiceUrl( parameter );
+    if( url != null ) {
+      Template template = Parser.parse( url );
+      Host host = template.getHost();
+      String hostStr = null;
+      if( host != null ) {
+        hostStr = host.getFirstValue().getPattern();
+        if( hostmap != null ) {
+          switch( direction ) {
+            case IN:
+              hostStr = hostmap.resolveInboundHostName( hostStr );
+              break;
+            case OUT:
+              hostStr = hostmap.resolveOutboundHostName( hostStr );
+              break;
+          }
+        }
+      }
+
+      Port port = template.getPort();
+      String portStr = null;
+      if( port != null ) {
+        portStr = port.getFirstValue().getPattern();
+      }
+
+      if( hostStr != null && portStr != null ) {
+        addr = hostStr + ":" + portStr;
+      } else if( host != null && port == null ) {
+        addr = hostStr;
+      } else if( host == null && port != null ) {
+        addr = ":" + portStr;
+      }
+    }
+    return addr;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedHostFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedHostFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedHostFunctionProcessor.java
new file mode 100644
index 0000000..d7ab283
--- /dev/null
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedHostFunctionProcessor.java
@@ -0,0 +1,87 @@
+/**
+ * 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.svcregfunc.impl;
+
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteEnvironment;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
+import org.apache.hadoop.gateway.services.GatewayServices;
+import org.apache.hadoop.gateway.services.hostmap.HostMapper;
+import org.apache.hadoop.gateway.services.hostmap.HostMapperService;
+import org.apache.hadoop.gateway.svcregfunc.api.ServiceMappedHostFunctionDescriptor;
+import org.apache.hadoop.gateway.util.urltemplate.Host;
+import org.apache.hadoop.gateway.util.urltemplate.Parser;
+import org.apache.hadoop.gateway.util.urltemplate.Template;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ServiceMappedHostFunctionProcessor
+    extends ServiceRegistryFunctionProcessorBase<ServiceMappedHostFunctionDescriptor>
+    implements UrlRewriteFunctionProcessor<ServiceMappedHostFunctionDescriptor> {
+
+  private HostMapper hostmap;
+
+  @Override
+  public String name() {
+    return ServiceMappedHostFunctionDescriptor.FUNCTION_NAME;
+  }
+
+  @Override
+  public void initialize( UrlRewriteEnvironment environment, ServiceMappedHostFunctionDescriptor descriptor ) throws Exception {
+    super.initialize( environment, descriptor );
+    HostMapperService hostmapService = services().getService( GatewayServices.HOST_MAPPING_SERVICE );
+    if( hostmapService != null ) {
+      hostmap = hostmapService.getHostMapper( cluster() );
+    }
+  }
+
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        String url = lookupServiceUrl( parameter );
+        if( url != null ) {
+          Template template = Parser.parse( url );
+          Host host = template.getHost();
+          if( host != null ) {
+            String hostStr = host.getFirstValue().getPattern();
+            if( hostmap != null ) {
+              switch( context.getDirection() ) {
+                case IN:
+                  parameter = hostmap.resolveInboundHostName( hostStr );
+                  break;
+                case OUT:
+                  parameter = hostmap.resolveOutboundHostName( hostStr );
+                  break;
+              }
+            } else {
+              parameter = hostStr;
+            }
+          }
+        }
+        results.add( parameter );
+      }
+    }
+    return results;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedUrlFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedUrlFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedUrlFunctionProcessor.java
new file mode 100644
index 0000000..2747681
--- /dev/null
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedUrlFunctionProcessor.java
@@ -0,0 +1,90 @@
+/**
+ * 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.svcregfunc.impl;
+
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteEnvironment;
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriter;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
+import org.apache.hadoop.gateway.services.GatewayServices;
+import org.apache.hadoop.gateway.services.hostmap.HostMapper;
+import org.apache.hadoop.gateway.services.hostmap.HostMapperService;
+import org.apache.hadoop.gateway.svcregfunc.api.ServiceMappedUrlFunctionDescriptor;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ServiceMappedUrlFunctionProcessor
+    extends ServiceRegistryFunctionProcessorBase<ServiceMappedUrlFunctionDescriptor>
+    implements UrlRewriteFunctionProcessor<ServiceMappedUrlFunctionDescriptor> {
+
+  private HostMapper hostmap;
+
+  @Override
+  public String name() {
+    return ServiceMappedUrlFunctionDescriptor.FUNCTION_NAME;
+  }
+
+  @Override
+  public void initialize( UrlRewriteEnvironment environment, ServiceMappedUrlFunctionDescriptor descriptor ) throws Exception {
+    super.initialize( environment, descriptor );
+    HostMapperService hostmapService = services().getService( GatewayServices.HOST_MAPPING_SERVICE );
+    if( hostmapService != null ) {
+      hostmap = hostmapService.getHostMapper( cluster() );
+    }
+  }
+
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      UrlRewriter.Direction direction = context.getDirection();
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        parameter = resolve( direction, parameter );
+        results.add( parameter );
+      }
+    }
+    return results;
+  }
+
+  public String resolve( UrlRewriter.Direction direction, String parameter ) throws Exception {
+    String url = lookupServiceUrl( parameter );
+    if( url != null ) {
+      URI outputUri;
+      URI inputUri = new URI( url );
+      String host = inputUri.getHost();
+      if( host != null && hostmap != null ) {
+        switch( direction ) {
+          case IN:
+            host = hostmap.resolveInboundHostName( host );
+            break;
+          case OUT:
+            host = hostmap.resolveOutboundHostName( host );
+            break;
+        }
+      }
+      outputUri = new URI( inputUri.getScheme(), inputUri.getUserInfo(), host, inputUri.getPort(), inputUri.getPath(), inputUri.getQuery(), inputUri.getFragment() );
+      parameter = outputUri.toString();
+    }
+    return parameter;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePathFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePathFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePathFunctionProcessor.java
index 4d2500d..d3a1229 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePathFunctionProcessor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePathFunctionProcessor.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.gateway.util.urltemplate.Parser;
 import org.apache.hadoop.gateway.util.urltemplate.Path;
 import org.apache.hadoop.gateway.util.urltemplate.Template;
 
+import java.util.ArrayList;
 import java.util.List;
 
 public class ServicePathFunctionProcessor
@@ -35,19 +36,24 @@ public class ServicePathFunctionProcessor
     return ServicePathFunctionDescriptor.FUNCTION_NAME;
   }
 
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    String value = parameter;
-    String url = super.resolve( context, parameter );
-    if( url != null && !url.equals(  parameter ) ) {
-      Template template = Parser.parse( url );
-      List<Path> path = template.getPath();
-      if( path != null ) {
-        value = toString( path );
-      } else {
-        value = parameter;
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        String url = lookupServiceUrl( parameter );
+        if( url != null ) {
+          Template template = Parser.parse( url );
+          List<Path> path = template.getPath();
+          if( path != null ) {
+            parameter = toString( path );
+          }
+        }
+        results.add( parameter );
       }
     }
-    return value;
+    return results;
   }
 
   private String toString( List<Path> paths ) {

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePortFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePortFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePortFunctionProcessor.java
index e4a5c79..7226922 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePortFunctionProcessor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServicePortFunctionProcessor.java
@@ -24,6 +24,9 @@ import org.apache.hadoop.gateway.util.urltemplate.Parser;
 import org.apache.hadoop.gateway.util.urltemplate.Port;
 import org.apache.hadoop.gateway.util.urltemplate.Template;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class ServicePortFunctionProcessor
     extends ServiceRegistryFunctionProcessorBase<ServicePortFunctionDescriptor>
     implements UrlRewriteFunctionProcessor<ServicePortFunctionDescriptor> {
@@ -33,17 +36,24 @@ public class ServicePortFunctionProcessor
     return ServicePortFunctionDescriptor.FUNCTION_NAME;
   }
 
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    String value = parameter;
-    String url = super.resolve( context, value );
-    if( url != null && !url.equals( parameter ) ) {
-      Template template = Parser.parse( url );
-      Port port = template.getPort();
-      if( port != null ) {
-        value = port.getFirstValue().getPattern();
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        String url = lookupServiceUrl( parameter );
+        if( url != null ) {
+          Template template = Parser.parse( url );
+          Port port = template.getPort();
+          if( port != null ) {
+            parameter = port.getFirstValue().getPattern();
+          }
+        }
+        results.add( parameter );
       }
     }
-    return value;
+    return results;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceRegistryFunctionProcessorBase.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceRegistryFunctionProcessorBase.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceRegistryFunctionProcessorBase.java
index 432a771..5e4479f 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceRegistryFunctionProcessorBase.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceRegistryFunctionProcessorBase.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.gateway.svcregfunc.impl;
 
 import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteEnvironment;
 import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor;
-import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
 import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
 import org.apache.hadoop.gateway.services.GatewayServices;
 import org.apache.hadoop.gateway.services.registry.ServiceRegistry;
@@ -27,6 +26,7 @@ import org.apache.hadoop.gateway.services.registry.ServiceRegistry;
 abstract class ServiceRegistryFunctionProcessorBase<T extends UrlRewriteFunctionDescriptor> implements UrlRewriteFunctionProcessor<T> {
 
   private String cluster;
+  private GatewayServices services;
   private ServiceRegistry registry;
 
   @Override
@@ -38,7 +38,7 @@ abstract class ServiceRegistryFunctionProcessorBase<T extends UrlRewriteFunction
     if( cluster == null ) {
       throw new IllegalArgumentException( "cluster==null" );
     }
-    GatewayServices services = environment.getAttribute( GatewayServices.GATEWAY_SERVICES_ATTRIBUTE );
+    services = environment.getAttribute( GatewayServices.GATEWAY_SERVICES_ATTRIBUTE );
     if( services == null ) {
       throw new IllegalArgumentException( "services==null" );
     }
@@ -54,20 +54,19 @@ abstract class ServiceRegistryFunctionProcessorBase<T extends UrlRewriteFunction
     cluster = null;
   }
 
-
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    String value = parameter;
-    String url = registry.lookupServiceURL( cluster, parameter );
-    if( url != null ) {
-      value = url;
-    }
-    return value;
+  public String lookupServiceUrl( String role ) throws Exception {
+    String url = registry.lookupServiceURL( cluster, role );
+    return url;
   }
 
   String cluster() {
     return cluster;
   }
 
+  GatewayServices services() {
+    return services;
+  }
+
   ServiceRegistry registry() {
     return registry;
   }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceSchemeFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceSchemeFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceSchemeFunctionProcessor.java
index 450df00..919f5c4 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceSchemeFunctionProcessor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceSchemeFunctionProcessor.java
@@ -24,6 +24,9 @@ import org.apache.hadoop.gateway.util.urltemplate.Parser;
 import org.apache.hadoop.gateway.util.urltemplate.Scheme;
 import org.apache.hadoop.gateway.util.urltemplate.Template;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class ServiceSchemeFunctionProcessor
     extends ServiceRegistryFunctionProcessorBase<ServiceSchemeFunctionDescriptor>
     implements UrlRewriteFunctionProcessor<ServiceSchemeFunctionDescriptor> {
@@ -33,17 +36,24 @@ public class ServiceSchemeFunctionProcessor
     return ServiceSchemeFunctionDescriptor.FUNCTION_NAME;
   }
 
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    String value = parameter;
-    String url = super.resolve( context, parameter );
-    if( url != null && !url.equals( parameter ) ) {
-      Template template = Parser.parse( url );
-      Scheme scheme = template.getScheme();
-      if( scheme != null ) {
-        value = scheme.getFirstValue().getPattern();
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        String url = lookupServiceUrl( parameter );
+        if( url != null && !url.equals( parameter ) ) {
+          Template template = Parser.parse( url );
+          Scheme scheme = template.getScheme();
+          if( scheme != null ) {
+            parameter = scheme.getFirstValue().getPattern();
+          }
+        }
+        results.add( parameter );
       }
     }
-    return value;
+    return results;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceUrlFunctionProcessor.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceUrlFunctionProcessor.java b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceUrlFunctionProcessor.java
index dd4ed0e..2adb3ff 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceUrlFunctionProcessor.java
+++ b/gateway-provider-rewrite-func-service-registry/src/main/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceUrlFunctionProcessor.java
@@ -17,10 +17,15 @@
  */
 package org.apache.hadoop.gateway.svcregfunc.impl;
 
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriter;
 import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
 import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
 import org.apache.hadoop.gateway.svcregfunc.api.ServiceUrlFunctionDescriptor;
 
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
 public class ServiceUrlFunctionProcessor
     extends ServiceRegistryFunctionProcessorBase<ServiceUrlFunctionDescriptor>
     implements UrlRewriteFunctionProcessor<ServiceUrlFunctionDescriptor> {
@@ -30,8 +35,30 @@ public class ServiceUrlFunctionProcessor
     return ServiceUrlFunctionDescriptor.FUNCTION_NAME;
   }
 
-  public String resolve( UrlRewriteContext context, String parameter ) throws Exception {
-    return super.resolve( context, parameter );
+  @Override
+  public List<String> resolve( UrlRewriteContext context, List<String> parameters ) throws Exception {
+    List<String> results = null;
+    if( parameters != null ) {
+      UrlRewriter.Direction direction = context.getDirection();
+      results = new ArrayList<String>( parameters.size() );
+      for( String parameter : parameters ) {
+        parameter = resolve( parameter );
+        results.add( parameter );
+      }
+    }
+    return results;
+  }
+
+  public String resolve( String parameter ) throws Exception {
+    String url = lookupServiceUrl( parameter );
+    if( url != null ) {
+      URI outputUri;
+      URI inputUri = new URI( url );
+      String host = inputUri.getHost();
+      outputUri = new URI( inputUri.getScheme(), inputUri.getUserInfo(), host, inputUri.getPort(), inputUri.getPath(), inputUri.getQuery(), inputUri.getFragment() );
+      parameter = outputUri.toString();
+    }
+    return parameter;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor b/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
index 8f50977..17adf92 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
+++ b/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteFunctionDescriptor
@@ -21,4 +21,7 @@ org.apache.hadoop.gateway.svcregfunc.api.ServiceAddressFunctionDescriptor
 org.apache.hadoop.gateway.svcregfunc.api.ServiceSchemeFunctionDescriptor
 org.apache.hadoop.gateway.svcregfunc.api.ServiceHostFunctionDescriptor
 org.apache.hadoop.gateway.svcregfunc.api.ServicePortFunctionDescriptor
-org.apache.hadoop.gateway.svcregfunc.api.ServicePathFunctionDescriptor
\ No newline at end of file
+org.apache.hadoop.gateway.svcregfunc.api.ServicePathFunctionDescriptor
+org.apache.hadoop.gateway.svcregfunc.api.ServiceMappedUrlFunctionDescriptor
+org.apache.hadoop.gateway.svcregfunc.api.ServiceMappedAddressFunctionDescriptor
+org.apache.hadoop.gateway.svcregfunc.api.ServiceMappedHostFunctionDescriptor

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor b/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor
index d241c10..498d286 100644
--- a/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor
+++ b/gateway-provider-rewrite-func-service-registry/src/main/resources/META-INF/services/org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor
@@ -21,4 +21,7 @@ org.apache.hadoop.gateway.svcregfunc.impl.ServiceAddressFunctionProcessor
 org.apache.hadoop.gateway.svcregfunc.impl.ServiceSchemeFunctionProcessor
 org.apache.hadoop.gateway.svcregfunc.impl.ServiceHostFunctionProcessor
 org.apache.hadoop.gateway.svcregfunc.impl.ServicePortFunctionProcessor
-org.apache.hadoop.gateway.svcregfunc.impl.ServicePathFunctionProcessor
\ No newline at end of file
+org.apache.hadoop.gateway.svcregfunc.impl.ServicePathFunctionProcessor
+org.apache.hadoop.gateway.svcregfunc.impl.ServiceMappedUrlFunctionProcessor
+org.apache.hadoop.gateway.svcregfunc.impl.ServiceMappedAddressFunctionProcessor
+org.apache.hadoop.gateway.svcregfunc.impl.ServiceMappedHostFunctionProcessor

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessorTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessorTest.java b/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessorTest.java
index 21f2bf5..5e99482 100644
--- a/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessorTest.java
+++ b/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceAddressFunctionProcessorTest.java
@@ -23,11 +23,11 @@ import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
 import org.apache.hadoop.gateway.services.GatewayServices;
 import org.apache.hadoop.gateway.services.registry.ServiceRegistry;
 import org.apache.hadoop.gateway.svcregfunc.api.ServiceAddressFunctionDescriptor;
-import org.apache.hadoop.gateway.svcregfunc.api.ServiceSchemeFunctionDescriptor;
 import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.ServiceLoader;
 
@@ -36,6 +36,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.CoreMatchers.sameInstance;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
 import static org.junit.Assert.fail;
 
 public class ServiceAddressFunctionProcessorTest {
@@ -124,8 +125,8 @@ public class ServiceAddressFunctionProcessorTest {
     ServiceAddressFunctionProcessor func = new ServiceAddressFunctionProcessor();
     func.initialize( env, desc );
 
-    assertThat( func.resolve( ctx, "test-service" ), is( "test-host:777" ) );
-    assertThat( func.resolve( ctx, "invalid-test-service" ), is( "invalid-test-service" ) );
+    assertThat( func.resolve( ctx, Arrays.asList( "test-service" ) ), contains( "test-host:777" ) );
+    assertThat( func.resolve( ctx, Arrays.asList( "invalid-test-service" ) ), contains( "invalid-test-service" ) );
     assertThat( func.resolve( ctx, null ), nullValue() );
 
     func.destroy();

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessorTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessorTest.java b/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessorTest.java
index 9544174..e9f2e6b 100644
--- a/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessorTest.java
+++ b/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceHostFunctionProcessorTest.java
@@ -22,13 +22,12 @@ import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
 import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
 import org.apache.hadoop.gateway.services.GatewayServices;
 import org.apache.hadoop.gateway.services.registry.ServiceRegistry;
-import org.apache.hadoop.gateway.svcregfunc.api.ServiceAddressFunctionDescriptor;
 import org.apache.hadoop.gateway.svcregfunc.api.ServiceHostFunctionDescriptor;
-import org.apache.hadoop.gateway.svcregfunc.api.ServiceSchemeFunctionDescriptor;
 import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.ServiceLoader;
 
@@ -37,6 +36,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.CoreMatchers.sameInstance;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
 import static org.junit.Assert.fail;
 
 public class ServiceHostFunctionProcessorTest {
@@ -125,8 +125,8 @@ public class ServiceHostFunctionProcessorTest {
     ServiceHostFunctionProcessor func = new ServiceHostFunctionProcessor();
     func.initialize( env, desc );
 
-    assertThat( func.resolve( ctx, "test-service" ), is( "test-host" ) );
-    assertThat( func.resolve( ctx, "invalid-test-service" ), is( "invalid-test-service" ) );
+    assertThat( func.resolve( ctx, Arrays.asList( "test-service" ) ), contains( "test-host" ) );
+    assertThat( func.resolve( ctx, Arrays.asList( "invalid-test-service" ) ), contains( "invalid-test-service" ) );
     assertThat( func.resolve( ctx, null ), nullValue() );
 
     func.destroy();

http://git-wip-us.apache.org/repos/asf/incubator-knox/blob/e338fe64/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessorTest.java
----------------------------------------------------------------------
diff --git a/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessorTest.java b/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessorTest.java
new file mode 100644
index 0000000..c1f2b06
--- /dev/null
+++ b/gateway-provider-rewrite-func-service-registry/src/test/java/org/apache/hadoop/gateway/svcregfunc/impl/ServiceMappedAddressFunctionProcessorTest.java
@@ -0,0 +1,148 @@
+/**
+ * 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.svcregfunc.impl;
+
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriteEnvironment;
+import org.apache.hadoop.gateway.filter.rewrite.api.UrlRewriter;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteContext;
+import org.apache.hadoop.gateway.filter.rewrite.spi.UrlRewriteFunctionProcessor;
+import org.apache.hadoop.gateway.services.GatewayServices;
+import org.apache.hadoop.gateway.services.hostmap.HostMapper;
+import org.apache.hadoop.gateway.services.hostmap.HostMapperService;
+import org.apache.hadoop.gateway.services.registry.ServiceRegistry;
+import org.apache.hadoop.gateway.svcregfunc.api.ServiceMappedAddressFunctionDescriptor;
+import org.easymock.EasyMock;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.ServiceLoader;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.CoreMatchers.sameInstance;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
+import static org.junit.Assert.fail;
+
+public class ServiceMappedAddressFunctionProcessorTest {
+
+  HostMapperService hms;
+  HostMapper hm;
+  ServiceRegistry reg;
+  GatewayServices svc;
+  UrlRewriteEnvironment env;
+  UrlRewriteContext ctx;
+  ServiceMappedAddressFunctionDescriptor desc;
+
+  @Before
+  public void setUp() {
+    hm = EasyMock.createNiceMock( HostMapper.class );
+    EasyMock.expect( hm.resolveInboundHostName( "test-host" ) ).andReturn( "test-internal-host" ).anyTimes();
+
+    hms = EasyMock.createNiceMock( HostMapperService.class );
+    EasyMock.expect( hms.getHostMapper( "test-cluster" ) ).andReturn( hm ).anyTimes();
+
+    reg = EasyMock.createNiceMock( ServiceRegistry.class );
+    EasyMock.expect( reg.lookupServiceURL( "test-cluster", "test-service" ) ).andReturn( "test-scheme://test-host:777/test-path" ).anyTimes();
+
+    svc = EasyMock.createNiceMock( GatewayServices.class );
+    EasyMock.expect( svc.getService( GatewayServices.SERVICE_REGISTRY_SERVICE ) ).andReturn( reg ).anyTimes();
+    EasyMock.expect( svc.getService( GatewayServices.HOST_MAPPING_SERVICE ) ).andReturn( hms ).anyTimes();
+
+    env = EasyMock.createNiceMock( UrlRewriteEnvironment.class );
+    EasyMock.expect( env.getAttribute( GatewayServices.GATEWAY_SERVICES_ATTRIBUTE ) ).andReturn( svc ).anyTimes();
+    EasyMock.expect( env.getAttribute( GatewayServices.GATEWAY_CLUSTER_ATTRIBUTE ) ).andReturn( "test-cluster" ).anyTimes();
+
+    ctx = EasyMock.createNiceMock( UrlRewriteContext.class );
+    EasyMock.expect( ctx.getDirection() ).andReturn( UrlRewriter.Direction.IN ).anyTimes();
+
+    desc = EasyMock.createNiceMock( ServiceMappedAddressFunctionDescriptor.class );
+
+    EasyMock.replay( hm, hms, reg, svc, env, desc, ctx );
+  }
+
+  @Test
+  public void testServiceLoader() throws Exception {
+    ServiceLoader loader = ServiceLoader.load( UrlRewriteFunctionProcessor.class );
+    Iterator iterator = loader.iterator();
+    assertThat( "Service iterator empty.", iterator.hasNext() );
+    while( iterator.hasNext() ) {
+      Object object = iterator.next();
+      if( object instanceof ServiceMappedAddressFunctionProcessor ) {
+        return;
+      }
+    }
+    fail( "Failed to find " + ServiceMappedAddressFunctionProcessor.class.getName() + " via service loader." );
+  }
+
+  @Test
+  public void testName() throws Exception {
+    ServiceMappedAddressFunctionProcessor func = new ServiceMappedAddressFunctionProcessor();
+    assertThat( func.name(), is( "serviceMappedAddr" ) );
+  }
+
+  @Test
+  public void testInitialize() throws Exception {
+    ServiceMappedAddressFunctionProcessor func = new ServiceMappedAddressFunctionProcessor();
+    try {
+      func.initialize( null, desc );
+      fail( "Should have thrown an IllegalArgumentException" );
+    } catch( IllegalArgumentException e ) {
+      assertThat( e.getMessage(), containsString( "environment" ) );
+    }
+
+    func = new ServiceMappedAddressFunctionProcessor();
+    try {
+      func.initialize( env, null );
+    } catch( Exception e ) {
+      e.printStackTrace();
+      fail( "Should not have thrown an exception" );
+    }
+
+    func.initialize( env, desc );
+
+    assertThat( func.cluster(), is( "test-cluster" ) );
+    assertThat( func.registry(), sameInstance( reg ) );
+  }
+
+  @Test
+  public void testDestroy() throws Exception {
+    ServiceMappedAddressFunctionProcessor func = new ServiceMappedAddressFunctionProcessor();
+    func.initialize( env, desc );
+    func.destroy();
+
+    assertThat( func.cluster(), nullValue() );
+    assertThat( func.registry(), nullValue() );
+  }
+
+  @Test
+  public void testResolve() throws Exception {
+    ServiceMappedAddressFunctionProcessor func = new ServiceMappedAddressFunctionProcessor();
+    func.initialize( env, desc );
+
+    assertThat( func.resolve( ctx, Arrays.asList( "test-service" ) ), contains( "test-internal-host:777" ) );
+    assertThat( func.resolve( ctx, Arrays.asList( "invalid-test-service" ) ), contains( "invalid-test-service" ) );
+    assertThat( func.resolve( ctx, null ), nullValue() );
+
+    func.destroy();
+  }
+
+}