You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2022/02/10 11:01:24 UTC

[GitHub] [hive] kgyrtkirk commented on a change in pull request #3006: HIVE-25575: Add support for JWT authentication in HTTP mode

kgyrtkirk commented on a change in pull request #3006:
URL: https://github.com/apache/hive/pull/3006#discussion_r803506170



##########
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##########
@@ -4046,6 +4046,12 @@ private static void populateLlapDaemonVarsSet(Set<String> llapDaemonVarsSetLocal
     HIVE_SERVER2_THRIFT_HTTP_COMPRESSION_ENABLED("hive.server2.thrift.http.compression.enabled", true,
         "Enable thrift http compression via Jetty compression support"),
 
+    // JWT Auth configs
+    HIVE_SERVER2_THRIFT_HTTP_JWT_JWKS_URL("hive.server2.thrift.http.jwt.jwks.url", "",

Review comment:
       from first glance this seem to be related to the `JWT` authentication; instead of placing it under this key; can we place it similarily to other authentication related conf keys?
   following the pattern of:
   ```
   hive.server2.authentication.ldap.url
   hive.server2.authentication.spnego.keytab
   hive.server2.authentication.pam.services
   ```
   I think this should better be: `hive.server2.authentication.jwt.url`

##########
File path: jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
##########
@@ -804,6 +811,31 @@ protected boolean requestIsAborted(final HttpRequest request) {
     return httpClientBuilder.build();
   }
 
+  private String getJWT() {
+    JWTFetcher jwtFetcher = new JWTFetcher();
+    String jwtCredential = jwtFetcher.getJWTStringFromSession();
+    if (jwtCredential == null) {
+      jwtCredential = jwtFetcher.getJWTStringFromEnv();
+    }
+    return jwtCredential;
+  }
+
+  class JWTFetcher {

Review comment:
       this doesn't look like a class; I think inlining these methods would lead to a more readable method

##########
File path: service/src/java/org/apache/hive/service/auth/jwt/HttpJwtAuthenticationException.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.auth.jwt;
+
+import org.apache.hive.service.auth.HttpAuthenticationException;
+
+/**
+ * Exception classes to be used to throw errors in the JWT authentication flow.
+ */
+public class HttpJwtAuthenticationException extends HttpAuthenticationException {

Review comment:
       unused class?

##########
File path: service/src/java/org/apache/hive/service/auth/jwt/JWKSProvider.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.auth.jwt;
+
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.jwk.JWK;
+
+import java.util.List;
+
+/**
+ * Provides a way to get JWKS json. Hive will use this to verify the incoming JWTs.
+ */
+public interface JWKSProvider {

Review comment:
       you don't have multiple implementations - so there is no point in having an interface either

##########
File path: service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
##########
@@ -737,6 +773,10 @@ private boolean isKerberosAuthMode(String authType) {
     return authType.equalsIgnoreCase(HiveAuthConstants.AuthTypes.KERBEROS.toString());
   }
 
+  private boolean isJWTAuthMode(String authType) {
+    return authType.toLowerCase().contains(HiveAuthConstants.AuthTypes.JWT.toString().toLowerCase());

Review comment:
       contains? I wonder what is the usecase here - so if `authType`  is `maybeJWTand` is also `JWT` auth?

##########
File path: service/src/java/org/apache/hive/service/auth/jwt/URLBasedJWKSProvider.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.auth.jwt;
+
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.jwk.JWK;
+import com.nimbusds.jose.jwk.JWKMatcher;
+import com.nimbusds.jose.jwk.JWKSelector;
+import com.nimbusds.jose.jwk.JWKSet;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link JWKSProvider} which reads JWKS from URL.
+ */
+public class URLBasedJWKSProvider implements JWKSProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(URLBasedJWKSProvider.class.getName());
+  private final HiveConf conf;
+  private List<JWKSet> jwkSets = new ArrayList<>();
+
+  public URLBasedJWKSProvider(HiveConf conf) {
+    this.conf = conf;
+    loadJWKSets();
+  }
+
+  private void loadJWKSets() {
+    String jwksURL = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_JWT_JWKS_URL);
+    List<String> jwksURLs = Arrays.stream(jwksURL.split(",")).collect(Collectors.toList());
+    for (String urlString : jwksURLs) {
+      try {
+        URL url = new URL(urlString);
+        jwkSets.add(JWKSet.load(url));
+        LOG.info("Loaded JWKS from " + urlString);
+      } catch (IOException | ParseException e) {
+        LOG.info("Failed to retrieve JWKS from {}: {}", urlString, e.getMessage());

Review comment:
       this is a bad design:
   * HS2 starts up
   * initializes the JWTValidator  which intializes this
     * which simply throws away exceptions ...
   * and could possibly reject auth requests because it didn't reported errors on startup...

##########
File path: service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
##########
@@ -301,6 +312,31 @@ protected void doPost(HttpServletRequest request, HttpServletResponse response)
     }
   }
 
+  private String validateJWT(HttpServletRequest request, HttpServletResponse response) {
+    Preconditions.checkState(jwtValidator != null, "JWT validator should have been set");
+    String signedJwt = extractBearerToken(request, response);
+    if (signedJwt == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No token found with the request " + request);
+      }
+      return null;
+    }
+    String user = null;
+    try {
+      user = jwtValidator.validateJWTAndExtractUser(signedJwt);
+      LOG.info("JWT verification successful for user - " + user);

Review comment:
       there will be a log message for success but nothing in case it failed....
   
   please use `{}` in log messages and stop using `if(LOG.isX())` when its not warranted
   ```
   LOG.info("JWT verification successful for user: {}", user);
   ```

##########
File path: itests/hive-unit/src/test/java/org/apache/hive/service/auth/jwt/TestHttpJwtAuthentication.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.auth.jwt;
+
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.JWSSigner;
+import com.nimbusds.jose.crypto.RSASSASigner;
+import com.nimbusds.jose.jwk.RSAKey;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.SignedJWT;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.jdbc.HiveConnection;
+import org.apache.hive.jdbc.Utils;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+
+public class TestHttpJwtAuthentication {
+  private static final Map<String, String> DEFAULTS = new HashMap<>(System.getenv());
+  private static Map<String, String> envMap;
+
+  private static final File jwtAuthorizedKeyFile =
+      new File("src/test/resources/auth.jwt/jwt-authorized-key.json");
+  private static final File jwtUnauthorizedKeyFile =
+      new File("src/test/resources/auth.jwt/jwt-unauthorized-key.json");
+  private static final File jwtVerificationJWKSFile =
+      new File("src/test/resources/auth.jwt/jwt-verification-jwks.json");
+
+  public static final String USER_1 = "USER_1";
+
+  private static MiniHS2 miniHS2;
+
+  @BeforeClass
+  public static void makeEnvModifiable() throws Exception {
+    envMap = new HashMap<>();
+    Class<?> envClass = Class.forName("java.lang.ProcessEnvironment");
+    Field theEnvironmentField = envClass.getDeclaredField("theEnvironment");
+    Field theUnmodifiableEnvironmentField = envClass.getDeclaredField("theUnmodifiableEnvironment");
+    removeStaticFinalAndSetValue(theEnvironmentField, envMap);
+    removeStaticFinalAndSetValue(theUnmodifiableEnvironmentField, envMap);
+  }
+
+  private static void removeStaticFinalAndSetValue(Field field, Object value) throws Exception {
+    field.setAccessible(true);
+    Field modifiersField = Field.class.getDeclaredField("modifiers");
+    modifiersField.setAccessible(true);
+    modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+    field.set(null, value);
+  }
+
+  @Before
+  public void initEnvMap() {
+    envMap.clear();
+    envMap.putAll(DEFAULTS);
+  }
+
+  @BeforeClass
+  public static void setupHS2() throws Exception {
+    HiveConf conf = new HiveConf();
+    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    conf.setBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED, false);
+    conf.setBoolVar(ConfVars.HIVESTATSCOLAUTOGATHER, false);
+    conf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, "JWT");
+    // the content of the URL below is the same as jwtVerificationJWKSFile
+    conf.setVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_JWT_JWKS_URL,
+        "https://gist.githubusercontent.com/hsnusonic/d06f2f18a73d1dbbba081e0267467da6/raw/38c2930d134c78320219b838bac4ceee680817bd/jwks.json");

Review comment:
       I think this will mean network usage during testing don't do that.
   and it also uses something which is out of our control

##########
File path: jdbc/src/java/org/apache/hive/jdbc/Utils.java
##########
@@ -99,6 +99,9 @@
     public static final String AUTH_PASSWD = "password";
     public static final String AUTH_KERBEROS_AUTH_TYPE = "kerberosAuthType";
     public static final String AUTH_KERBEROS_AUTH_TYPE_FROM_SUBJECT = "fromSubject";
+    public static final String AUTH_TYPE_JWT = "jwt";
+    public static final String AUTH_TYPE_JWT_KEY = "jwt";
+    public static final String AUTH_JWT_ENV = "JWT";

Review comment:
       is the name of this environment variable is the recommended; or it was arbitrarily choosen ?

##########
File path: service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
##########
@@ -155,9 +164,10 @@ protected void doPost(HttpServletRequest request, HttpServletResponse response)
 
       clientIpAddress = request.getRemoteAddr();
       LOG.debug("Client IP Address: " + clientIpAddress);
+
       // If the cookie based authentication is already enabled, parse the
       // request and validate the request cookies.
-      if (isCookieAuthEnabled) {
+      if (clientUserName == null && isCookieAuthEnabled) {

Review comment:
       at this point `clientUserName == null` is alwas true

##########
File path: service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
##########
@@ -203,29 +213,32 @@ protected void doPost(HttpServletRequest request, HttpServletResponse response)
           // HTTP request header.
           clientUserName = doPasswdAuth(request, HiveAuthConstants.AuthTypes.NOSASL.getAuthName());
         } else {
+          if (isJWTAuthMode(authType)) {
+            clientUserName = validateJWT(request, response);
+          }
           // For a kerberos setup
-          if (isKerberosAuthMode(authType)) {
+          if (clientUserName == null && isKerberosAuthMode(authType)) {

Review comment:
       why add all these `clientUserName == null` checks ? if its JWT auth and it failed - don't try to fallback to others; I don't understand why would that be preferrable




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org