You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2021/12/17 08:00:47 UTC

[GitHub] [hbase] taklwu commented on a change in pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

taklwu commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r771039262



##########
File path: hbase-client/src/test/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientCallbackHandlerTest.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.hbase.security.provider;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import java.io.IOException;
+import java.security.AccessController;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
+import java.util.Set;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil;
+import org.junit.Test;
+
+public class OAuthBearerSaslClientCallbackHandlerTest {
+  private static OAuthBearerToken createTokenWithLifetimeMillis(final long lifetimeMillis) {
+    return new OAuthBearerToken() {
+      @Override
+      public String value() {
+        return null;
+      }
+
+      @Override
+      public String principalName() {
+        return null;
+      }
+
+      @Override
+      public long lifetimeMs() {
+        return lifetimeMillis;
+      }
+    };
+  }
+
+  @Test
+  public void testWithZeroTokens() {
+    OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler handler =
+      createCallbackHandler();
+    PrivilegedActionException e =
+      assertThrows(PrivilegedActionException.class, () -> Subject.doAs(new Subject(),
+      (PrivilegedExceptionAction<Void>) () -> {
+          OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+          handler.handle(new Callback[] {callback});
+          return null;
+        }
+    ));
+    assertEquals(IOException.class, e.getCause().getClass());
+  }
+
+  @Test()

Review comment:
       [nit] remove `()`?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerTokenCallback.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.hbase.security.oauthbearer;
+
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslClient} and {@code Login}
+ * implementations when they require an OAuth 2 bearer token. Callback handlers
+ * should use the {@link #error(String, String, String)} method to communicate
+ * errors returned by the authorization server as per
+ * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+ * 2.0 Authorization Framework</a>. Callback handlers should communicate other
+ * problems by raising an {@code IOException}.
+ * <p>
+ * This class was introduced in 2.0.0 and, while it feels stable, it could

Review comment:
       [nit] should it be 3.0.0? or are you referring to package other than hbase? 

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/Utils.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.hbase.security.oauthbearer;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Public
+public final class Utils {
+  /**
+   *  Converts a {@code Map} class into a string, concatenating keys and values
+   *  Example:
+   *      {@code mkString({ key: "hello", keyTwo: "hi" }, "|START|", "|END|", "=", ",")
+   *          => "|START|key=hello,keyTwo=hi|END|"}
+   */
+  public static <K, V> String mkString(Map<K, V> map, String begin, String end,
+    String keyValueSeparator, String elementSeparator) {
+    StringBuilder bld = new StringBuilder();
+    bld.append(begin);
+    String prefix = "";
+    for (Map.Entry<K, V> entry : map.entrySet()) {
+      bld.append(prefix).append(entry.getKey()).
+        append(keyValueSeparator).append(entry.getValue());
+      prefix = elementSeparator;
+    }
+    bld.append(end);
+    return bld.toString();
+  }
+
+  /**
+   *  Converts an extensions string into a {@code Map<String, String>}.
+   *
+   *  Example:
+   *      {@code parseMap("key=hey,keyTwo=hi,keyThree=hello", "=", ",") =>
+   *      { key: "hey", keyTwo: "hi", keyThree: "hello" }}
+   *
+   */
+  public static Map<String, String> parseMap(String mapStr,
+    String keyValueSeparator, String elementSeparator) {
+    Map<String, String> map = new HashMap<>();
+
+    if (!mapStr.isEmpty()) {
+      String[] attrvals = mapStr.split(elementSeparator);
+      for (String attrval : attrvals) {
+        String[] array = attrval.split(keyValueSeparator, 2);
+        map.put(array[0], array[1]);
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Given two maps (A, B), returns all the key-value pairs in A whose keys are not contained in B
+   */
+  public static <K, V> Map<K, V> subtractMap(Map<? extends K, ? extends V> minuend,
+    Map<? extends K, ? extends V> subtrahend) {
+    return minuend.entrySet().stream()
+      .filter(entry -> !subtrahend.containsKey(entry.getKey()))
+      .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+  }
+
+  /**
+   * Checks if a string is null, empty or whitespace only.
+   * @param str a string to be checked
+   * @return true if the string is null, empty or whitespace only; otherwise, return false.
+   */
+  public static boolean isBlank(String str) {
+    return str == null || str.trim().isEmpty();
+  }

Review comment:
       [nit] can we use `StringUtils.isBlank` instead?

##########
File path: hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/JwtTestUtils.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.hbase.security.oauthbearer;
+
+import com.nimbusds.jose.JOSEException;
+import com.nimbusds.jose.JOSEObjectType;
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jose.JWSHeader;
+import com.nimbusds.jose.crypto.RSASSASigner;
+import com.nimbusds.jose.jwk.RSAKey;
+import com.nimbusds.jose.jwk.gen.RSAKeyGenerator;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.SignedJWT;
+import java.util.Date;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Public
+public final class JwtTestUtils {
+  public static final long ONE_DAY = 24 * 60 * 60 * 1000L;

Review comment:
       [nit] maybe use `TimeUnit.DAYS.toMillis()`?
   
   ```suggestion
     public static final long ONE_DAY = 24 * 60 * 60 * 1000L;
   ```
   ```suggestion
     public static final long ONE_DAY = TimeUnit.DAYS.toMillis(1L);
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwtValidatorCallbackHandler.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.hbase.security.oauthbearer.internals.knox;
+
+import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.OAUTHBEARER_MECHANISM;
+import com.nimbusds.jose.jwk.JWKSet;
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.text.ParseException;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.Utils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@code CallbackHandler} that recognizes
+ * {@link OAuthBearerValidatorCallback} and validates a secure (signed) OAuth 2
+ * bearer token (JWT).
+ *
+ * It requires a valid JWK Set to be initialized at startup which holds the available
+ * RSA public keys that JWT signature can be validated with. The Set can be initialized
+ * via an URL or a local file.
+ *
+ * It requires there to be an <code>"exp" (Expiration Time)</code>
+ * claim of type Number. If <code>"iat" (Issued At)</code> or
+ * <code>"nbf" (Not Before)</code> claims are present each must be a number that
+ * precedes the Expiration Time claim, and if both are present the Not Before
+ * claim must not precede the Issued At claim. It also accepts the following
+ * options, none of which are required:
+ * <ul>
+ * <li>{@code hbase.security.oauth.jwt.jwks.url} set to a non-empty value if you
+ * wish to initialize the JWK Set via an URL. HTTPS URLs must have valid certificates.
+ * </li>
+ * <li>{@code hbase.security.oauth.jwt.jwks.file} set to a non-empty value if you
+ * wish to initialize the JWK Set from a local JSON file.
+ * </li>
+ * <li>{@code hbase.security.oauth.jwt.audience} set to a String value which
+ * you want the desired audience ("aud") the JWT to have.</li>
+ * <li>{@code hbase.security.oauth.jwt.issuer} set to a String value which
+ * you want the issuer ("iss") of the JWT has to be.</li>
+ * <li>{@code hbase.security.oauth.jwt.allowableclockskewseconds} set to a positive integer
+ * value if you wish to allow up to some number of positive seconds of
+ * clock skew (the default is 0)</li>
+ * </ul>
+ *
+ * It also recognizes {@link OAuthBearerExtensionsValidatorCallback} and validates
+ * every extension passed to it.
+ *
+ * This class is based on Kafka's OAuthBearerUnsecuredValidatorCallbackHandler.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerSignedJwtValidatorCallbackHandler implements AuthenticateCallbackHandler {
+  private static final Logger LOG =
+    LoggerFactory.getLogger(OAuthBearerSignedJwtValidatorCallbackHandler.class);
+  private static final String OPTION_PREFIX = "hbase.security.oauth.jwt.";
+  private static final String JWKS_URL = OPTION_PREFIX + "jwks.url";
+  private static final String JWKS_FILE = OPTION_PREFIX + "jwks.file";
+  private static final String ALLOWABLE_CLOCK_SKEW_SECONDS_OPTION =
+    OPTION_PREFIX + "allowableclockskewseconds";
+  static final String REQUIRED_AUDIENCE_OPTION = OPTION_PREFIX + "audience";
+  static final String REQUIRED_ISSUER_OPTION = OPTION_PREFIX + "issuer";
+  private Configuration hBaseConfiguration;
+  private JWKSet jwkSet;
+  private boolean configured = false;
+
+  @Override
+  public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+    if (!configured) {
+      throw new RuntimeException(
+        "OAuthBearerSignedJwtValidatorCallbackHandler handler be configured first.");
+    }

Review comment:
       just wondered if this runtime exception is trying to protect us from malicious attack ? or if checking the `jwkSet` isn't null not enough ? 

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.hbase.exceptions;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This exception indicates unexpected requests prior to SASL authentication.
+ * This could be due to misconfigured security, e.g. if PLAINTEXT protocol
+ * is used to connect to a SASL endpoint.
+ */
+@InterfaceAudience.Public
+public class IllegalSaslStateException extends IllegalStateException {
+
+  private static final long serialVersionUID = 1L;
+
+  public IllegalSaslStateException(String message) {
+    super(message);
+  }
+
+  public IllegalSaslStateException(String message, Throwable cause) {

Review comment:
       [nit] seems to be an unused function ?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/Utils.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.hbase.security.oauthbearer;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Public
+public final class Utils {

Review comment:
       [nit] maybe name it as `OAuthStringUtils` or `OAuthBearerStringUtils`? it just seems strange to me this only call `Utils` 

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.hbase.security.oauthbearer.internals;
+
+import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.OAUTHBEARER_MECHANISM;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import org.apache.hadoop.hbase.exceptions.SaslAuthenticationException;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.Utils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@code SaslServer} implementation for SASL/OAUTHBEARER in Kafka. An instance
+ * of {@link OAuthBearerToken} is available upon successful authentication via
+ * the negotiated property "{@code OAUTHBEARER.token}"; the token could be used
+ * in a custom authorizer (to authorize based on JWT claims rather than ACLs,
+ * for example).
+ */
+@InterfaceAudience.Public
+public class OAuthBearerSaslServer implements SaslServer {
+  public static final Logger LOG = LoggerFactory.getLogger(OAuthBearerSaslServer.class);
+  private static final String NEGOTIATED_PROPERTY_KEY_TOKEN = OAUTHBEARER_MECHANISM + ".token";
+  private static final String INTERNAL_ERROR_ON_SERVER =
+    "Authentication could not be performed due to an internal error on the server";
+  static final String CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY =
+    "CREDENTIAL.LIFETIME.MS";
+
+  private final AuthenticateCallbackHandler callbackHandler;
+
+  private boolean complete;
+  private OAuthBearerToken tokenForNegotiatedProperty = null;
+  private String errorMessage = null;
+  private SaslExtensions extensions;
+
+  public OAuthBearerSaslServer(CallbackHandler callbackHandler) {
+    if (!(Objects.requireNonNull(callbackHandler) instanceof AuthenticateCallbackHandler)) {
+      throw new IllegalArgumentException(
+        String.format("Callback handler must be castable to %s: %s",
+          AuthenticateCallbackHandler.class.getName(), callbackHandler.getClass().getName()));
+    }
+    this.callbackHandler = (AuthenticateCallbackHandler) callbackHandler;
+  }
+
+  /**
+   * @throws SaslAuthenticationException
+   *             if access token cannot be validated
+   *             <p>
+   *             <b>Note:</b> This method may throw
+   *             {@link SaslAuthenticationException} to provide custom error
+   *             messages to clients. But care should be taken to avoid including
+   *             any information in the exception message that should not be
+   *             leaked to unauthenticated clients. It may be safer to throw
+   *             {@link SaslException} in some cases so that a standard error
+   *             message is returned to clients.
+   *             </p>
+   */
+  @Override
+  public byte[] evaluateResponse(byte[] response)
+    throws SaslException, SaslAuthenticationException {
+    try {
+      if (response.length == 1 && response[0] == OAuthBearerSaslClient.BYTE_CONTROL_A &&
+        errorMessage != null) {
+        LOG.error("Received %x01 response from client after it received our error");
+        throw new SaslAuthenticationException(errorMessage);
+      }
+      errorMessage = null;

Review comment:
       [nit] do we need this line ? 
   ```suggestion
         errorMessage = null;
   ```
   ```suggestion
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.hbase.security.auth;
+
+import java.util.Map;
+import javax.security.auth.callback.CallbackHandler;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/*
+ * Callback handler for SASL-based authentication
+ */
+@InterfaceAudience.Public
+public interface AuthenticateCallbackHandler extends CallbackHandler {
+
+  /**
+   * Configures this callback handler for the specified SASL mechanism.
+   *
+   * @param configs Key-value pairs containing the parsed configuration options of
+   *        the client or server. Note that these are the HBase configuration options
+   *        and not the JAAS configuration options. JAAS config options may be obtained
+   *        from `jaasConfigEntries` for callbacks which obtain some configs from the
+   *        JAAS configuration. For configs that may be specified as both HBase config
+   *        as well as JAAS config (e.g. sasl.kerberos.service.name), the configuration
+   *        is treated as invalid if conflicting values are provided.
+   * @param saslMechanism Negotiated SASL mechanism. For clients, this is the SASL
+   *        mechanism configured for the client. For brokers, this is the mechanism
+   *        negotiated with the client and is one of the mechanisms enabled on the broker.
+   * @param saslProps SASL properties provided by the SASL library.
+   */
+  default void configure(
+    Configuration configs, String saslMechanism, Map<String, String> saslProps) {}
+
+  /**
+   * Closes this instance.
+   */
+  default void close() {}

Review comment:
       [nit] can we use `Closeable`? and I didn't find a non-empty use case with this close() yet in this change. 

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.hbase.security.oauthbearer.internals;
+
+import static org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.OAUTHBEARER_MECHANISM;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import org.apache.hadoop.hbase.exceptions.SaslAuthenticationException;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerExtensionsValidatorCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.Utils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@code SaslServer} implementation for SASL/OAUTHBEARER in Kafka. An instance
+ * of {@link OAuthBearerToken} is available upon successful authentication via
+ * the negotiated property "{@code OAUTHBEARER.token}"; the token could be used
+ * in a custom authorizer (to authorize based on JWT claims rather than ACLs,
+ * for example).
+ */
+@InterfaceAudience.Public
+public class OAuthBearerSaslServer implements SaslServer {
+  public static final Logger LOG = LoggerFactory.getLogger(OAuthBearerSaslServer.class);
+  private static final String NEGOTIATED_PROPERTY_KEY_TOKEN = OAUTHBEARER_MECHANISM + ".token";
+  private static final String INTERNAL_ERROR_ON_SERVER =
+    "Authentication could not be performed due to an internal error on the server";
+  static final String CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY =
+    "CREDENTIAL.LIFETIME.MS";
+
+  private final AuthenticateCallbackHandler callbackHandler;
+
+  private boolean complete;
+  private OAuthBearerToken tokenForNegotiatedProperty = null;
+  private String errorMessage = null;
+  private SaslExtensions extensions;
+
+  public OAuthBearerSaslServer(CallbackHandler callbackHandler) {
+    if (!(Objects.requireNonNull(callbackHandler) instanceof AuthenticateCallbackHandler)) {
+      throw new IllegalArgumentException(
+        String.format("Callback handler must be castable to %s: %s",
+          AuthenticateCallbackHandler.class.getName(), callbackHandler.getClass().getName()));
+    }
+    this.callbackHandler = (AuthenticateCallbackHandler) callbackHandler;
+  }
+
+  /**
+   * @throws SaslAuthenticationException
+   *             if access token cannot be validated
+   *             <p>
+   *             <b>Note:</b> This method may throw
+   *             {@link SaslAuthenticationException} to provide custom error
+   *             messages to clients. But care should be taken to avoid including
+   *             any information in the exception message that should not be
+   *             leaked to unauthenticated clients. It may be safer to throw
+   *             {@link SaslException} in some cases so that a standard error
+   *             message is returned to clients.
+   *             </p>
+   */
+  @Override
+  public byte[] evaluateResponse(byte[] response)
+    throws SaslException, SaslAuthenticationException {
+    try {
+      if (response.length == 1 && response[0] == OAuthBearerSaslClient.BYTE_CONTROL_A &&
+        errorMessage != null) {
+        LOG.error("Received %x01 response from client after it received our error");
+        throw new SaslAuthenticationException(errorMessage);
+      }
+      errorMessage = null;
+
+      OAuthBearerClientInitialResponse clientResponse;
+      clientResponse = new OAuthBearerClientInitialResponse(response);
+
+      return process(clientResponse.tokenValue(), clientResponse.authorizationId(),
+        clientResponse.extensions());
+    } catch (SaslAuthenticationException e) {
+      LOG.error("SASL authentication error: {}", e.getMessage());

Review comment:
       [nit] don't we want to print the full stacktrace in the code? 
   ```suggestion
         LOG.error("SASL authentication error: {}", e.getMessage());
   ```
   ```suggestion
         LOG.error("SASL authentication error: {}", e);
   ```

##########
File path: hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallbackTest.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.hbase.security.oauthbearer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import org.junit.Test;
+
+public class OAuthBearerValidatorCallbackTest {

Review comment:
       I'm wondered why the build didn't fail without the `@Category` and `CLASS_RULE` , but anyway, can you add them back to all test classes?

##########
File path: hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java
##########
@@ -0,0 +1,106 @@
+/**
+ * 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.hbase.jwt.client.example;
+
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An example of using OAuthBearer (JWT) authentication with HBase RPC client.
+ */
+@InterfaceAudience.Private
+public class JwtClientExample extends Configured implements Tool {
+  private static final Logger LOG = LoggerFactory.getLogger(JwtClientExample.class);
+  private static final String jwt = "<base64_encoded_jwt_token>";
+
+  private static final byte[] FAMILY = Bytes.toBytes("d");
+
+  public JwtClientExample() {
+    setConf(HBaseConfiguration.create());
+  }

Review comment:
       as an example code, should we set the providers ? but I think you're assuming me the hbase-site has them, right?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
##########
@@ -365,7 +365,7 @@ public void saslReadAndProcess(ByteBuff saslToken) throws IOException,
             throw e;
           }
           RpcServer.LOG.debug("Created SASL server with mechanism={}",
-              provider.getSaslAuthMethod().getAuthMethod());
+              provider.getSaslAuthMethod().getSaslMechanism());

Review comment:
       nice catch

##########
File path: hbase-examples/src/main/java/org/apache/hadoop/hbase/jwt/client/example/JwtClientExample.java
##########
@@ -0,0 +1,106 @@
+/**
+ * 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.hbase.jwt.client.example;
+
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An example of using OAuthBearer (JWT) authentication with HBase RPC client.
+ */
+@InterfaceAudience.Private
+public class JwtClientExample extends Configured implements Tool {
+  private static final Logger LOG = LoggerFactory.getLogger(JwtClientExample.class);
+  private static final String jwt = "<base64_encoded_jwt_token>";

Review comment:
       ```suggestion
     private static final String jwt = "<base64_encoded_jwt_token>";
   ```
   ```suggestion
     private static final String JWT_TOKEN = "<base64_encoded_jwt_token>";
   ```




-- 
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: issues-unsubscribe@hbase.apache.org

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