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 2014/03/26 05:25:30 UTC

git commit: KNOX-337: Knox not authenticating with HBase 0.98 in secure mode (cherry picked from commit 49eb750c4b9d7b14482bbb158f0dc91b29d46aa2)

Repository: knox
Updated Branches:
  refs/heads/master 0e1df0f87 -> dd93ed27a


KNOX-337: Knox not authenticating with HBase 0.98 in secure mode
(cherry picked from commit 49eb750c4b9d7b14482bbb158f0dc91b29d46aa2)


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

Branch: refs/heads/master
Commit: dd93ed27a0c6744a8fe5a57952847ad8bfb65bb0
Parents: 0e1df0f
Author: Kevin Minder <ke...@hortonworks.com>
Authored: Wed Mar 26 00:24:51 2014 -0400
Committer: Kevin Minder <ke...@hortonworks.com>
Committed: Wed Mar 26 00:25:18 2014 -0400

----------------------------------------------------------------------
 .../gateway/hbase/HBaseCookieManager.java       | 43 +++++++++++++
 .../hbase/HBaseDeploymentContributor.java       |  4 +-
 .../HBaseDispatchDeploymentContributor.java     | 64 ++++++++++++++++++++
 .../gateway/hbase/HBaseHttpClientDispatch.java  | 37 +++++++++++
 ...gateway.deploy.ProviderDeploymentContributor | 19 ++++++
 .../HBaseDispatchDeploymentContributorTest.java | 45 ++++++++++++++
 .../gateway/dispatch/AppCookieManager.java      | 10 ++-
 .../gateway/dispatch/HttpClientDispatch.java    | 13 ++--
 8 files changed, 227 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseCookieManager.java
----------------------------------------------------------------------
diff --git a/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseCookieManager.java b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseCookieManager.java
new file mode 100644
index 0000000..8eea445
--- /dev/null
+++ b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseCookieManager.java
@@ -0,0 +1,43 @@
+/**
+ * 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.hbase;
+
+import org.apache.hadoop.gateway.dispatch.AppCookieManager;
+import org.apache.http.HttpRequest;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpUriRequest;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+public class HBaseCookieManager extends AppCookieManager {
+
+  protected HttpRequest createKerberosAuthenticationRequest( HttpUriRequest userRequest ) {
+    URI userUri = userRequest.getURI();
+    try {
+      URI authUri = new URI(
+          userUri.getScheme(), null, userUri.getHost(), userUri.getPort(),
+          "/version", userUri.getQuery(), null );
+      HttpRequest authRequest = new HttpGet( authUri );
+      return authRequest;
+    } catch( URISyntaxException e ) {
+      throw new IllegalArgumentException( userUri.toString(), e );
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDeploymentContributor.java b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDeploymentContributor.java
index 0e6915f..336f42e 100644
--- a/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDeploymentContributor.java
+++ b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDeploymentContributor.java
@@ -114,8 +114,8 @@ public class HBaseDeploymentContributor extends ServiceDeploymentContributorBase
   }
 
   private void addDispatchFilter(
-      DeploymentContext context, Service service, ResourceDescriptor resource ) {
-    context.contributeFilter( service, resource, "dispatch", "http-client", null );
+    DeploymentContext context, Service service, ResourceDescriptor resource ) {
+    context.contributeFilter( service, resource, "dispatch", "hbase", null );
   }
 
   private String getQualifiedName() {

http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributor.java
----------------------------------------------------------------------
diff --git a/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributor.java b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributor.java
new file mode 100644
index 0000000..3eb24cb
--- /dev/null
+++ b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributor.java
@@ -0,0 +1,64 @@
+/**
+ * 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.hbase;
+
+import org.apache.hadoop.gateway.deploy.DeploymentContext;
+import org.apache.hadoop.gateway.deploy.ProviderDeploymentContributorBase;
+import org.apache.hadoop.gateway.descriptor.FilterDescriptor;
+import org.apache.hadoop.gateway.descriptor.FilterParamDescriptor;
+import org.apache.hadoop.gateway.descriptor.ResourceDescriptor;
+import org.apache.hadoop.gateway.topology.Provider;
+import org.apache.hadoop.gateway.topology.Service;
+
+import java.util.List;
+
+public class HBaseDispatchDeploymentContributor extends ProviderDeploymentContributorBase {
+  
+  private static final String REPLAY_BUFFER_SIZE_PARAM = "replayBufferSize";
+  
+  // Default global replay buffer size in KB
+  public static final String DEFAULT_REPLAY_BUFFER_SIZE = "4";
+
+  @Override
+  public String getRole() {
+    return "dispatch";
+  }
+
+  @Override
+  public String getName() {
+    return "hbase";
+  }
+
+  @Override
+  public void contributeFilter( DeploymentContext context, Provider provider, Service service, ResourceDescriptor resource, List<FilterParamDescriptor> params ) {
+    String replayBufferSize = DEFAULT_REPLAY_BUFFER_SIZE;
+    if (params != null) {
+      for (FilterParamDescriptor paramDescriptor : params) {
+        if (REPLAY_BUFFER_SIZE_PARAM.equals( paramDescriptor.name() )) {
+          replayBufferSize = paramDescriptor.value();
+          break;
+        }
+      }
+    }
+    FilterDescriptor filter = resource.addFilter().name( getName() ).role( getRole() ).impl( HBaseHttpClientDispatch.class );
+    filter.param().name("replayBufferSize").value(replayBufferSize);
+    if( context.getGatewayConfig().isHadoopKerberosSecured() ) {
+      filter.param().name("kerberos").value("true");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseHttpClientDispatch.java
----------------------------------------------------------------------
diff --git a/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseHttpClientDispatch.java b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseHttpClientDispatch.java
new file mode 100644
index 0000000..a569692
--- /dev/null
+++ b/gateway-service-hbase/src/main/java/org/apache/hadoop/gateway/hbase/HBaseHttpClientDispatch.java
@@ -0,0 +1,37 @@
+/**
+ * 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.hbase;
+
+import org.apache.hadoop.gateway.dispatch.HttpClientDispatch;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+
+/**
+ * This specialized dispatch provides HBase specific features to the
+ * default HttpClientDispatch.
+ */
+public class HBaseHttpClientDispatch extends HttpClientDispatch {
+
+  @Override
+  public void init( FilterConfig filterConfig ) throws ServletException {
+    super.init( filterConfig, new HBaseCookieManager() );
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-service-hbase/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor
----------------------------------------------------------------------
diff --git a/gateway-service-hbase/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor b/gateway-service-hbase/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor
new file mode 100644
index 0000000..610bfbb
--- /dev/null
+++ b/gateway-service-hbase/src/main/resources/META-INF/services/org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor
@@ -0,0 +1,19 @@
+##########################################################################
+# 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.
+##########################################################################
+
+org.apache.hadoop.gateway.hbase.HBaseDispatchDeploymentContributor

http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-service-hbase/src/test/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributorTest.java
----------------------------------------------------------------------
diff --git a/gateway-service-hbase/src/test/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributorTest.java b/gateway-service-hbase/src/test/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributorTest.java
new file mode 100644
index 0000000..e2b4d10
--- /dev/null
+++ b/gateway-service-hbase/src/test/java/org/apache/hadoop/gateway/hbase/HBaseDispatchDeploymentContributorTest.java
@@ -0,0 +1,45 @@
+/**
+ * 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.hbase;
+
+import org.apache.hadoop.gateway.deploy.ProviderDeploymentContributor;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.ServiceLoader;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+public class HBaseDispatchDeploymentContributorTest {
+
+  @Test
+  public void testServiceLoader() throws Exception {
+    ServiceLoader loader = ServiceLoader.load( ProviderDeploymentContributor.class );
+    Iterator iterator = loader.iterator();
+    assertThat( "Service iterator empty.", iterator.hasNext() );
+    while( iterator.hasNext() ) {
+      Object object = iterator.next();
+      if( object instanceof HBaseDispatchDeploymentContributor ) {
+        return;
+      }
+    }
+    fail( "Failed to find " + HBaseDispatchDeploymentContributor.class.getName() + " via service loader." );
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/AppCookieManager.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/AppCookieManager.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/AppCookieManager.java
index f8b687c..cbbbcc5 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/AppCookieManager.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/AppCookieManager.java
@@ -43,6 +43,7 @@ import org.apache.http.client.methods.HttpUriRequest;
 import org.apache.http.client.params.AuthPolicy;
 import org.apache.http.impl.auth.SPNegoSchemeFactory;
 import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.util.EntityUtils;
 
 /**
  * Handles SPNego authentication as a client of hadoop service, caches
@@ -95,7 +96,6 @@ public class AppCookieManager {
     String scheme = uri.getScheme();
     String host = uri.getHost();
     int port = uri.getPort();
-    String path = uri.getPath();
     if (!refresh) {
       if (appCookie != null) {
         return appCookie;
@@ -116,10 +116,11 @@ public class AppCookieManager {
     HttpResponse httpResponse = null;
     try {
       HttpHost httpHost = new HttpHost(host, port, scheme);
-      HttpRequest httpRequest = new HttpOptions(path);
+      HttpRequest httpRequest = createKerberosAuthenticationRequest( outboundRequest );
       httpResponse = client.execute(httpHost, httpRequest);
       Header[] headers = httpResponse.getHeaders(SET_COOKIE);
       hadoopAuthCookie = getHadoopAuthCookieValue(headers);
+      EntityUtils.consume( httpResponse.getEntity() );
       if (hadoopAuthCookie == null) {
         LOG.failedSPNegoAuthn(uri.toString());
         auditor.audit( Action.AUTHENTICATION, uri.toString(), ResourceType.URI, ActionOutcome.FAILURE );
@@ -142,6 +143,11 @@ public class AppCookieManager {
     return appCookie;
   }
 
+  protected HttpRequest createKerberosAuthenticationRequest( HttpUriRequest userRequest ) {
+    HttpRequest authRequest = new HttpOptions( userRequest.getURI().getPath() );
+    return authRequest;
+  }
+
   /**
    * Returns the cached app cookie
    * 

http://git-wip-us.apache.org/repos/asf/knox/blob/dd93ed27/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/HttpClientDispatch.java
----------------------------------------------------------------------
diff --git a/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/HttpClientDispatch.java b/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/HttpClientDispatch.java
index a85d880..c1b9625 100644
--- a/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/HttpClientDispatch.java
+++ b/gateway-spi/src/main/java/org/apache/hadoop/gateway/dispatch/HttpClientDispatch.java
@@ -76,21 +76,26 @@ public class HttpClientDispatch extends AbstractGatewayDispatch {
           AuditConstants.KNOX_SERVICE_NAME, AuditConstants.KNOX_COMPONENT_NAME );
   private static final int DEFAULT_REPLAY_BUFFER_SIZE =  4 * 1024; // 4K
 
-  protected AppCookieManager appCookieManager = new AppCookieManager();
+  protected AppCookieManager appCookieManager;
   
   protected static final String REPLAY_BUFFER_SIZE_PARAM = "replayBufferSize";
   
   private int replayBufferSize = 0;
-  
+
   @Override
   public void init( FilterConfig filterConfig ) throws ServletException {
-    super.init(filterConfig);
+    this.init(filterConfig, new AppCookieManager() );
+  }
+
+  protected void init( FilterConfig filterConfig, AppCookieManager cookieManager ) throws ServletException {
+    super.init( filterConfig );
+    appCookieManager = cookieManager;
     String replayBufferSizeString = filterConfig.getInitParameter( REPLAY_BUFFER_SIZE_PARAM );
     if ( replayBufferSizeString != null ) {
       setReplayBufferSize(Integer.valueOf(replayBufferSizeString));
     }
   }
-  
+
   protected void executeRequest(
       HttpUriRequest outboundRequest,
       HttpServletRequest inboundRequest,