You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sz...@apache.org on 2018/02/20 19:16:25 UTC

hive git commit: HIVE-18541 : Secure HS2 web UI with PAM (Oleksiy Sayankin via Szehon)

Repository: hive
Updated Branches:
  refs/heads/master e05e0fa19 -> 03a1e6247


HIVE-18541 : Secure HS2 web UI with PAM	(Oleksiy Sayankin via Szehon)


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

Branch: refs/heads/master
Commit: 03a1e624760f15c57bed04cba9ff6f3a5f1846c4
Parents: e05e0fa
Author: Szehon Ho <sz...@gmail.com>
Authored: Tue Feb 20 20:12:59 2018 +0100
Committer: Szehon Ho <sz...@gmail.com>
Committed: Tue Feb 20 20:12:59 2018 +0100

----------------------------------------------------------------------
 common/pom.xml                                  |  20 ++
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   2 +
 .../java/org/apache/hive/http/HttpServer.java   |  45 ++++-
 .../hive/http/security/PamAuthenticator.java    | 140 ++++++++++++++
 .../hive/http/security/PamConstraint.java       |  34 ++++
 .../http/security/PamConstraintMapping.java     |  27 +++
 .../hive/http/security/PamLoginService.java     |  78 ++++++++
 .../hive/http/security/PamUserIdentity.java     |  38 ++++
 .../apache/hive/service/server/HiveServer2.java |  25 +++
 .../service/server/TestHS2HttpServerPam.java    | 182 +++++++++++++++++++
 .../TestHS2HttpServerPamConfiguration.java      | 128 +++++++++++++
 11 files changed, 718 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index aaeecc0..4da46f2 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -79,6 +79,11 @@
     </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-http</artifactId>
+      <version>${jetty.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-rewrite</artifactId>
     </dependency>
     <dependency>
@@ -160,6 +165,21 @@
       </exclusions>
    </dependency>
     <dependency>
+      <groupId>net.sf.jpam</groupId>
+      <artifactId>jpam</artifactId>
+      <version>${jpam.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commmons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-core</artifactId>
       <version>${hadoop.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 3d777f9..b51dc7e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2450,6 +2450,8 @@ public class HiveConf extends Configuration {
         "the value of hive.server2.webui.host or the correct host name."),
     HIVE_SERVER2_WEBUI_MAX_HISTORIC_QUERIES("hive.server2.webui.max.historic.queries", 25,
         "The maximum number of past queries to show in HiverSever2 WebUI."),
+    HIVE_SERVER2_WEBUI_USE_PAM("hive.server2.webui.use.pam", false,
+        "If true, the HiveServer2 WebUI will be secured with PAM."),
 
     // Tez session settings
     HIVE_SERVER2_TEZ_INTERACTIVE_QUEUE("hive.server2.tez.interactive.queue", "",

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/src/java/org/apache/hive/http/HttpServer.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/HttpServer.java b/common/src/java/org/apache/hive/http/HttpServer.java
index 2a8f7ae..71b2668 100644
--- a/common/src/java/org/apache/hive/http/HttpServer.java
+++ b/common/src/java/org/apache/hive/http/HttpServer.java
@@ -21,6 +21,7 @@ package org.apache.hive.http;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URL;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -45,6 +46,10 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
+import org.apache.hive.http.security.PamAuthenticator;
+import org.apache.hive.http.security.PamConstraint;
+import org.apache.hive.http.security.PamConstraintMapping;
+import org.apache.hive.http.security.PamLoginService;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.Logger;
@@ -54,7 +59,11 @@ import org.apache.logging.log4j.core.appender.FileManager;
 import org.apache.logging.log4j.core.appender.OutputStreamManager;
 import org.eclipse.jetty.rewrite.handler.RewriteHandler;
 import org.eclipse.jetty.rewrite.handler.RewriteRegexRule;
+import org.eclipse.jetty.security.ConstraintMapping;
+import org.eclipse.jetty.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.security.LoginService;
 import org.eclipse.jetty.server.Connector;
+import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.HttpConfiguration;
 import org.eclipse.jetty.server.HttpConnectionFactory;
 import org.eclipse.jetty.server.LowResourceMonitor;
@@ -68,6 +77,7 @@ import org.eclipse.jetty.servlet.FilterMapping;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.security.Constraint;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.eclipse.jetty.util.thread.QueuedThreadPool;
 import org.eclipse.jetty.webapp.WebAppContext;
@@ -114,6 +124,8 @@ public class HttpServer {
     private String spnegoKeytab;
     private boolean useSPNEGO;
     private boolean useSSL;
+    private boolean usePAM;
+    private PamAuthenticator pamAuthenticator;
     private String contextRootRewriteTarget = "/index.html";
     private final List<Pair<String, Class<? extends HttpServlet>>> servlets =
         new LinkedList<Pair<String, Class<? extends HttpServlet>>>();
@@ -172,6 +184,16 @@ public class HttpServer {
       return this;
     }
 
+    public Builder setUsePAM(boolean usePAM) {
+      this.usePAM = usePAM;
+      return this;
+    }
+
+    public Builder setPAMAuthenticator(PamAuthenticator pamAuthenticator){
+      this.pamAuthenticator = pamAuthenticator;
+      return this;
+    }
+
     public Builder setUseSPNEGO(boolean useSPNEGO) {
       this.useSPNEGO = useSPNEGO;
       return this;
@@ -388,6 +410,22 @@ public class HttpServer {
   }
 
   /**
+   * Secure the web server with PAM.
+   */
+  void setupPam(Builder b, Handler handler) {
+    LoginService loginService = new PamLoginService();
+    webServer.addBean(loginService);
+    ConstraintSecurityHandler security = new ConstraintSecurityHandler();
+    Constraint constraint = new PamConstraint();
+    ConstraintMapping mapping = new PamConstraintMapping(constraint);
+    security.setConstraintMappings(Collections.singletonList(mapping));
+    security.setAuthenticator(b.pamAuthenticator);
+    security.setLoginService(loginService);
+    security.setHandler(handler);
+    webServer.setHandler(security);
+  }
+
+  /**
    * Set servlet context attributes that can be used in jsp.
    */
   void setContextAttributes(Context ctx, Map<String, Object> contextAttrs) {
@@ -417,7 +455,7 @@ public class HttpServer {
     initializeWebServer(b, threadPool.getMaxThreads());
   }
 
-  private void initializeWebServer(final Builder b, int queueSize) {
+  private void initializeWebServer(final Builder b, int queueSize) throws IOException {
     // Set handling for low resource conditions.
     final LowResourceMonitor low = new LowResourceMonitor(webServer);
     low.setLowResourcesIdleTimeout(10000);
@@ -443,6 +481,11 @@ public class HttpServer {
     contexts.addHandler(rwHandler);
     webServer.setHandler(contexts);
 
+    if(b.usePAM){
+      setupPam(b, contexts);
+    }
+
+
     addServlet("jmx", "/jmx", JMXJsonServlet.class);
     addServlet("conf", "/conf", ConfServlet.class);
     addServlet("stacks", "/stacks", StackServlet.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/src/java/org/apache/hive/http/security/PamAuthenticator.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/security/PamAuthenticator.java b/common/src/java/org/apache/hive/http/security/PamAuthenticator.java
new file mode 100644
index 0000000..cbc19dd
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/security/PamAuthenticator.java
@@ -0,0 +1,140 @@
+/*
+ * 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.hive.http.security;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.eclipse.jetty.http.HttpHeader;
+import org.eclipse.jetty.security.ServerAuthException;
+import org.eclipse.jetty.security.UserAuthentication;
+import org.eclipse.jetty.security.authentication.DeferredAuthentication;
+import org.eclipse.jetty.security.authentication.LoginAuthenticator;
+import org.eclipse.jetty.server.Authentication;
+import org.eclipse.jetty.server.UserIdentity;
+import org.eclipse.jetty.util.B64Code;
+
+import javax.security.sasl.AuthenticationException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import net.sf.jpam.Pam;
+
+/*
+
+  This class authenticates HS2 web UI via PAM. To authenticate use
+
+   * httpGet with header name "Authorization"
+   * and header value "Basic authB64Code"
+
+    where  authB64Code is Base64 string for "login:password"
+ */
+
+public class PamAuthenticator extends LoginAuthenticator {
+  private final String pamServiceNames;
+
+  public PamAuthenticator(HiveConf conf) throws AuthenticationException {
+    super();
+    pamServiceNames = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PAM_SERVICES);
+    if (pamServiceNames == null || pamServiceNames.trim().isEmpty()) {
+      throw new AuthenticationException("No PAM services are set.");
+    }
+  }
+
+  @Override
+  public String getAuthMethod() {
+    return "pam";
+  }
+
+  @Override
+  public Authentication validateRequest(ServletRequest req, ServletResponse res, boolean mandatory)
+      throws ServerAuthException {
+    HttpServletRequest request = (HttpServletRequest) req;
+    HttpServletResponse response = (HttpServletResponse) res;
+    String credentials = request.getHeader(HttpHeader.AUTHORIZATION.asString());
+
+    try {
+      if (!mandatory)
+        return new DeferredAuthentication(this);
+
+      if (credentials != null) {
+        int space = credentials.indexOf(' ');
+        if (space > 0) {
+          String method = credentials.substring(0, space);
+          if ("basic".equalsIgnoreCase(method)) {
+            credentials = credentials.substring(space + 1);
+            credentials = B64Code.decode(credentials, StandardCharsets.ISO_8859_1);
+            int i = credentials.indexOf(':');
+            if (i > 0) {
+              String username = credentials.substring(0, i);
+              String password = credentials.substring(i + 1);
+
+              UserIdentity user = login(username, password);
+              if (user != null) {
+                return new UserAuthentication(getAuthMethod(), user);
+              }
+            }
+          }
+        }
+      }
+
+      if (DeferredAuthentication.isDeferred(response))
+        return Authentication.UNAUTHENTICATED;
+
+      response.setHeader(HttpHeader.WWW_AUTHENTICATE.asString(), "basic realm=\"" + _loginService.getName() + '"');
+      response.sendError(HttpServletResponse.SC_UNAUTHORIZED);
+      return Authentication.SEND_CONTINUE;
+    } catch (IOException e) {
+      throw new ServerAuthException(e);
+    }
+  }
+
+  protected UserIdentity login(String username, String password) throws AuthenticationException {
+    UserIdentity user = null;
+    if (authenticate(username, password)) {
+      user = new PamUserIdentity(username);
+    }
+    return user;
+  }
+
+  private boolean authenticate(String user, String password) throws AuthenticationException {
+    String[] pamServices = pamServiceNames.split(",");
+    String errorMsg = "Error authenticating with the PAM service: ";
+    for (String pamService : pamServices) {
+      try {
+        Pam pam = new Pam(pamService);
+        if (!pam.authenticateSuccessful(user, password)) {
+          return false;
+        }
+      } catch (Throwable e) {
+        // Catch the exception caused by missing jpam.so which otherwise would
+        // crashes the thread and causes the client hanging rather than notifying
+        // the client nicely
+        throw new AuthenticationException(errorMsg + pamService, e);
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public boolean secureResponse(ServletRequest servletRequest, ServletResponse servletResponse, boolean b,
+      Authentication.User user) throws ServerAuthException {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/src/java/org/apache/hive/http/security/PamConstraint.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/security/PamConstraint.java b/common/src/java/org/apache/hive/http/security/PamConstraint.java
new file mode 100644
index 0000000..35ccdad
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/security/PamConstraint.java
@@ -0,0 +1,34 @@
+/*
+ * 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.hive.http.security;
+
+import org.eclipse.jetty.util.security.Constraint;
+
+public class PamConstraint extends Constraint {
+  private static final String[] roles = { "pam" };
+
+  @Override
+  public boolean getAuthenticate() {
+    return true;
+  }
+
+  @Override
+  public String[] getRoles() {
+    return roles;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/src/java/org/apache/hive/http/security/PamConstraintMapping.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/security/PamConstraintMapping.java b/common/src/java/org/apache/hive/http/security/PamConstraintMapping.java
new file mode 100644
index 0000000..489e724
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/security/PamConstraintMapping.java
@@ -0,0 +1,27 @@
+/*
+ * 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.hive.http.security;
+
+import org.eclipse.jetty.security.ConstraintMapping;
+import org.eclipse.jetty.util.security.Constraint;
+
+public class PamConstraintMapping extends ConstraintMapping {
+  public PamConstraintMapping(Constraint constraint) {
+    setPathSpec("/*");
+    setConstraint(constraint);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/src/java/org/apache/hive/http/security/PamLoginService.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/security/PamLoginService.java b/common/src/java/org/apache/hive/http/security/PamLoginService.java
new file mode 100644
index 0000000..8cf725a
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/security/PamLoginService.java
@@ -0,0 +1,78 @@
+/*
+ * 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.hive.http.security;
+
+import org.eclipse.jetty.security.DefaultIdentityService;
+import org.eclipse.jetty.security.IdentityService;
+import org.eclipse.jetty.security.LoginService;
+import org.eclipse.jetty.server.UserIdentity;
+import org.eclipse.jetty.util.component.AbstractLifeCycle;
+import org.eclipse.jetty.util.log.Log;
+import org.eclipse.jetty.util.log.Logger;
+
+import javax.servlet.ServletRequest;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class PamLoginService extends AbstractLifeCycle implements LoginService {
+  private final ConcurrentMap<String, UserIdentity> users = new ConcurrentHashMap<>();
+
+  private IdentityService identityService = new DefaultIdentityService();
+
+  private static final Logger LOG = Log.getLogger(PamLoginService.class);
+
+  @Override
+  public String getName() {
+    return "pam";
+  }
+
+  @Override
+  public UserIdentity login(String username, Object credentials, ServletRequest request) {
+    UserIdentity user = users.get(username);
+
+    if (user != null) {
+      return user;
+    }
+
+    user = new PamUserIdentity(username);
+    users.put(username, user);
+    return user;
+  }
+
+  @Override
+  public boolean validate(UserIdentity user) {
+    return users.containsKey(user.getUserPrincipal().getName());
+  }
+
+  @Override
+  public IdentityService getIdentityService() {
+    return identityService;
+  }
+
+  @Override
+  public void setIdentityService(IdentityService identityService) {
+    if (isRunning())
+      throw new IllegalStateException("Running");
+    this.identityService = identityService;
+  }
+
+  @Override
+  public void logout(UserIdentity user) {
+    users.remove(user.getUserPrincipal().getName());
+    LOG.debug("logout {}", user);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/common/src/java/org/apache/hive/http/security/PamUserIdentity.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/http/security/PamUserIdentity.java b/common/src/java/org/apache/hive/http/security/PamUserIdentity.java
new file mode 100644
index 0000000..85f074d
--- /dev/null
+++ b/common/src/java/org/apache/hive/http/security/PamUserIdentity.java
@@ -0,0 +1,38 @@
+/*
+ * 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.hive.http.security;
+
+import com.sun.security.auth.UserPrincipal;
+import org.eclipse.jetty.security.DefaultUserIdentity;
+
+import javax.security.auth.Subject;
+
+public class PamUserIdentity extends DefaultUserIdentity {
+  public PamUserIdentity(String username) {
+    super(new Subject(), new UserPrincipal(username), new String[] { "pam" });
+  }
+
+  @Override
+  public boolean isUserInRole(String role, Scope scope) {
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return PamUserIdentity.class.getSimpleName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 2e57b60..6c1a0b9 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -78,6 +78,7 @@ import org.apache.hive.common.util.HiveVersionInfo;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.apache.hive.http.HttpServer;
 import org.apache.hive.http.LlapServlet;
+import org.apache.hive.http.security.PamAuthenticator;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.ServiceException;
 import org.apache.hive.service.cli.CLIService;
@@ -116,12 +117,21 @@ public class HiveServer2 extends CompositeService {
   private HttpServer webServer; // Web UI
   private TezSessionPoolManager tezSessionPoolManager;
   private WorkloadManager wm;
+  private PamAuthenticator pamAuthenticator;
 
   public HiveServer2() {
     super(HiveServer2.class.getSimpleName());
     HiveConf.setLoadHiveServer2Config(true);
   }
 
+  @VisibleForTesting
+  public HiveServer2(PamAuthenticator pamAuthenticator) {
+    super(HiveServer2.class.getSimpleName());
+    HiveConf.setLoadHiveServer2Config(true);
+    this.pamAuthenticator = pamAuthenticator;
+  }
+
+
   @Override
   public synchronized void init(HiveConf hiveConf) {
     //Initialize metrics first, as some metrics are for initialization stuff.
@@ -255,6 +265,21 @@ public class HiveServer2 extends CompositeService {
             builder.setSPNEGOKeytab(spnegoKeytab);
             builder.setUseSPNEGO(true);
           }
+          if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_PAM)) {
+            if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_SSL)) {
+              String hiveServer2PamServices = hiveConf.getVar(ConfVars.HIVE_SERVER2_PAM_SERVICES);
+              if (hiveServer2PamServices == null || hiveServer2PamServices.isEmpty()) {
+                throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_PAM_SERVICES.varname + " are not configured.");
+              }
+              builder.setPAMAuthenticator(pamAuthenticator == null ? new PamAuthenticator(hiveConf) : pamAuthenticator);
+              builder.setUsePAM(true);
+            } else if (hiveConf.getBoolVar(ConfVars.HIVE_IN_TEST)) {
+              builder.setPAMAuthenticator(pamAuthenticator == null ? new PamAuthenticator(hiveConf) : pamAuthenticator);
+              builder.setUsePAM(true);
+            } else {
+              throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_WEBUI_USE_SSL.varname + " has false value. It is recommended to set to true when PAM is used.");
+            }
+          }
           builder.addServlet("llap", LlapServlet.class);
           builder.setContextRootRewriteTarget("/hiveserver2.jsp");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java
new file mode 100644
index 0000000..d1b3ce0
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPam.java
@@ -0,0 +1,182 @@
+/*
+ * 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.hive.service.server;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hive.http.security.PamAuthenticator;
+import org.apache.hive.http.security.PamUserIdentity;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.eclipse.jetty.http.HttpHeader;
+import org.eclipse.jetty.server.UserIdentity;
+import org.eclipse.jetty.util.B64Code;
+import org.eclipse.jetty.util.StringUtil;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.security.sasl.AuthenticationException;
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.security.cert.CertificateFactory;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * TestHS2HttpServerPam -- executes tests of HiveServer2 HTTP Server for Pam authentication
+ */
+public class TestHS2HttpServerPam {
+
+  private static HiveServer2 hiveServer2 = null;
+  private static HiveConf hiveConf = null;
+  private static String metastorePasswd = "693efe9fa425ad21886d73a0fa3fbc70"; //random md5
+  private static Integer webUIPort = null;
+  private static String host = "localhost";
+
+  @BeforeClass
+  public static void beforeTests() throws Exception {
+    webUIPort =
+        MetaStoreTestUtils.findFreePortExcepting(Integer.valueOf(ConfVars.HIVE_SERVER2_WEBUI_PORT.getDefaultValue()));
+    hiveConf = new HiveConf();
+    hiveConf.setBoolVar(ConfVars.HIVE_IN_TEST, true);
+    hiveConf.set(ConfVars.METASTOREPWD.varname, metastorePasswd);
+    hiveConf.set(ConfVars.HIVE_SERVER2_WEBUI_PORT.varname, webUIPort.toString());
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+        "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_SERVER2_PAM_SERVICES, "sshd");
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_PAM, true);
+    hiveConf.setBoolVar(ConfVars.HIVE_IN_TEST, true);
+    hiveServer2 = new HiveServer2(new TestPamAuthenticator(hiveConf));
+    hiveServer2.init(hiveConf);
+    hiveServer2.start();
+    Thread.sleep(5000);
+  }
+
+  @Test
+  public void testUnauthorizedConnection() throws Exception {
+    String baseURL = "http://" + host + ":" + webUIPort + "/stacks";
+    URL url = new URL(baseURL);
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
+  }
+
+  @Test
+  public void testAuthorizedConnection() throws Exception {
+    CloseableHttpClient httpclient = null;
+    try {
+      String username = "user1";
+      String password = "1";
+      httpclient = HttpClients.createDefault();
+
+      HttpGet httpGet = new HttpGet("http://" + host + ":" + webUIPort);
+      String authB64Code = B64Code.encode(username + ":" + password, StringUtil.__ISO_8859_1);
+      httpGet.setHeader(HttpHeader.AUTHORIZATION.asString(), "Basic " + authB64Code);
+      CloseableHttpResponse response = httpclient.execute(httpGet);
+      Assert.assertTrue(response.toString().contains(Integer.toString(HttpURLConnection.HTTP_OK)));
+
+    } finally {
+      if (httpclient != null) {
+        httpclient.close();
+      }
+    }
+  }
+
+  @Test
+  public void testIncorrectUser() throws Exception {
+    CloseableHttpClient httpclient = null;
+    try {
+      String username = "nouser";
+      String password = "aaaa";
+      httpclient = HttpClients.createDefault();
+
+      HttpGet httpGet = new HttpGet("http://" + host + ":" + webUIPort);
+      String authB64Code = B64Code.encode(username + ":" + password, StringUtil.__ISO_8859_1);
+      httpGet.setHeader(HttpHeader.AUTHORIZATION.asString(), "Basic " + authB64Code);
+      CloseableHttpResponse response = httpclient.execute(httpGet);
+      Assert.assertTrue(response.toString().contains(Integer.toString(HttpURLConnection.HTTP_UNAUTHORIZED)));
+
+    } finally {
+      if (httpclient != null) {
+        httpclient.close();
+      }
+    }
+  }
+
+  @Test
+  public void testIncorrectPassword() throws Exception {
+    CloseableHttpClient httpclient = null;
+    try {
+      String username = "user1";
+      String password = "aaaa";
+      httpclient = HttpClients.createDefault();
+
+      HttpGet httpGet = new HttpGet("http://" + host + ":" + webUIPort);
+      String authB64Code = B64Code.encode(username + ":" + password, StringUtil.__ISO_8859_1);
+      httpGet.setHeader(HttpHeader.AUTHORIZATION.asString(), "Basic " + authB64Code);
+      CloseableHttpResponse response = httpclient.execute(httpGet);
+      Assert.assertTrue(response.toString().contains(Integer.toString(HttpURLConnection.HTTP_UNAUTHORIZED)));
+
+    } finally {
+      if (httpclient != null) {
+        httpclient.close();
+      }
+    }
+  }
+
+  public static class TestPamAuthenticator extends PamAuthenticator {
+    private static final Map<String, String> users = new HashMap<>();
+
+    TestPamAuthenticator(HiveConf conf) throws AuthenticationException {
+      super(conf);
+    }
+
+    static {
+      users.put("user1", "1");
+      users.put("user2", "2");
+      users.put("user3", "3");
+      users.put("user4", "4");
+    }
+
+    @Override protected UserIdentity login(String username, String password) {
+      if (users.containsKey(username)) {
+        if (users.get(username).equals(password)) {
+          return new PamUserIdentity(username);
+        }
+      }
+      return null;
+    }
+  }
+
+  @AfterClass public static void afterTests() {
+    hiveServer2.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/03a1e624/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java
new file mode 100644
index 0000000..8d978cd
--- /dev/null
+++ b/service/src/test/org/apache/hive/service/server/TestHS2HttpServerPamConfiguration.java
@@ -0,0 +1,128 @@
+/*
+ * 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.hive.service.server;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.hamcrest.CoreMatchers.is;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.cert.CertificateException;
+
+/**
+ * TestHS2HttpServerPamConfiguration -- checks configuration for HiveServer2 HTTP Server with Pam authentication
+ */
+public class TestHS2HttpServerPamConfiguration {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private static HiveServer2 hiveServer2 = null;
+  private static HiveConf hiveConf = null;
+  private static String keyStorePassword = "123456";
+  private static String keyFileName = "myKeyStore";
+  private static String testDataDir = new File(
+      System.getProperty("java.io.tmpdir") + File.separator + TestHS2HttpServerPam.class.getCanonicalName() + "-"
+          + System.currentTimeMillis()).getPath().replaceAll("\\\\", "/");
+  private static String sslKeyStorePath = testDataDir + File.separator + keyFileName;
+
+
+  @BeforeClass
+  public static void beforeTests() throws Exception {
+    createTestDir();
+    createDefaultKeyStore();
+    String metastorePasswd = "693efe9fa425ad21886d73a0fa3fbc70"; //random md5
+    Integer webUIPort =
+        MetaStoreTestUtils.findFreePortExcepting(Integer.valueOf(ConfVars.HIVE_SERVER2_WEBUI_PORT.getDefaultValue()));
+    hiveConf = new HiveConf();
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_PAM, true);
+    hiveConf.setBoolVar(ConfVars.HIVE_IN_TEST, false);
+    hiveConf.set(ConfVars.METASTOREPWD.varname, metastorePasswd);
+    hiveConf.set(ConfVars.HIVE_SERVER2_WEBUI_PORT.varname, webUIPort.toString());
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+        "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+  }
+
+  @Test
+  public void testSslIsFalse() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(is(ConfVars.HIVE_SERVER2_WEBUI_USE_SSL.varname
+        + " has false value. It is recommended to set to true when PAM is used."));
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_PAM_SERVICES, "sshd");
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_SSL, false);
+    hiveServer2 = new HiveServer2();
+    hiveServer2.init(hiveConf);
+  }
+
+  @Test
+  public void testPamServicesAreNotConfigured() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(is(ConfVars.HIVE_SERVER2_PAM_SERVICES.varname + " are not configured."));
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_PAM_SERVICES, "");
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_SSL, true);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_WEBUI_SSL_KEYSTORE_PATH, sslKeyStorePath);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_WEBUI_SSL_KEYSTORE_PASSWORD, keyStorePassword);
+    hiveServer2 = new HiveServer2();
+    hiveServer2.init(hiveConf);
+  }
+
+  @Test
+  public void testPamCorrectConfiguration() {
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_PAM_SERVICES, "sshd");
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_WEBUI_USE_SSL, true);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_WEBUI_SSL_KEYSTORE_PATH, sslKeyStorePath);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_WEBUI_SSL_KEYSTORE_PASSWORD, keyStorePassword);
+    hiveServer2 = new HiveServer2();
+    hiveServer2.init(hiveConf);
+  }
+
+  @AfterClass
+  public static void afterTests() throws IOException {
+    FileUtils.deleteDirectory(new File(testDataDir));
+  }
+
+  private static void createTestDir() {
+    if (!(new File(testDataDir).mkdirs())) {
+      throw new RuntimeException("Could not create " + testDataDir);
+    }
+  }
+
+  private static void createDefaultKeyStore()
+      throws KeyStoreException, CertificateException, NoSuchAlgorithmException, IOException {
+    KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
+    char[] password = keyStorePassword.toCharArray();
+    ks.load(null, null);
+
+    // Store away the keystore.
+    try (FileOutputStream fos = new FileOutputStream(sslKeyStorePath)) {
+      ks.store(fos, password);
+    }
+  }
+}