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/10 11:19:05 UTC

[GitHub] [hbase] anmolnar opened a new pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

anmolnar opened a new pull request #3934:
URL: https://github.com/apache/hbase/pull/3934


   Initial commit for JWT authentication.
   
   Details are in the Jira: https://issues.apache.org/jira/browse/HBASE-26553


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778136979



##########
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());
+      throw e;
+    } catch (Exception e) {
+      LOG.error("SASL server problem", e);
+      throw e;
+    }
+  }
+
+  @Override
+  public String getAuthorizationID() {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return tokenForNegotiatedProperty.principalName();
+  }
+
+  @Override
+  public String getMechanismName() {
+    return OAUTHBEARER_MECHANISM;
+  }
+
+  @Override
+  public Object getNegotiatedProperty(String propName) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    if (NEGOTIATED_PROPERTY_KEY_TOKEN.equals(propName)) {
+      return tokenForNegotiatedProperty;
+    }
+    if (CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY.equals(propName)) {
+      return tokenForNegotiatedProperty.lifetimeMs();
+    }
+    return extensions.map().get(propName);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return complete;
+  }
+
+  @Override
+  public byte[] unwrap(byte[] incoming, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(incoming, offset, offset + len);
+  }
+
+  @Override
+  public byte[] wrap(byte[] outgoing, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(outgoing, offset, offset + len);
+  }
+
+  @Override
+  public void dispose() {
+    complete = false;
+    tokenForNegotiatedProperty = null;
+    extensions = null;
+  }
+
+  private byte[] process(String tokenValue, String authorizationId, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(tokenValue);
+    try {
+      callbackHandler.handle(new Callback[] {callback});
+    } catch (IOException | UnsupportedCallbackException e) {
+      handleCallbackError(e);
+    }
+    OAuthBearerToken token = callback.token();
+    if (token == null) {
+      errorMessage = jsonErrorResponse(callback.errorStatus(), callback.errorScope(),
+        callback.errorOpenIDConfiguration());
+      LOG.error("JWT token validation error: {}", errorMessage);
+      return errorMessage.getBytes(StandardCharsets.UTF_8);
+    }
+    /*
+     * We support the client specifying an authorization ID as per the SASL
+     * specification, but it must match the principal name if it is specified.
+     */
+    if (!authorizationId.isEmpty() && !authorizationId.equals(token.principalName())) {
+      throw new SaslAuthenticationException(String.format(
+        "Authentication failed: Client requested an authorization id (%s) that is different from "
+          + "the token's principal name (%s)",
+        authorizationId, token.principalName()));
+    }
+
+    Map<String, String> validExtensions = processExtensions(token, extensions);
+
+    tokenForNegotiatedProperty = token;
+    this.extensions = new SaslExtensions(validExtensions);
+    complete = true;
+    LOG.debug("Successfully authenticate User={}", token.principalName());
+    return new byte[0];
+  }
+
+  private Map<String, String> processExtensions(OAuthBearerToken token, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerExtensionsValidatorCallback
+      extensionsCallback = new OAuthBearerExtensionsValidatorCallback(token, extensions);
+    try {
+      callbackHandler.handle(new Callback[] {extensionsCallback});
+    } catch (UnsupportedCallbackException e) {
+      // backwards compatibility - no extensions will be added
+    } catch (IOException e) {
+      handleCallbackError(e);
+    }
+    if (!extensionsCallback.invalidExtensions().isEmpty()) {
+      String errorMessage = String.format("Authentication failed: %d extensions are invalid! "
+          + "They are: %s", extensionsCallback.invalidExtensions().size(),
+        Utils.mkString(extensionsCallback.invalidExtensions(), "", "", ": ", "; "));
+      LOG.debug(errorMessage);
+      throw new SaslAuthenticationException(errorMessage);
+    }
+
+    return extensionsCallback.validatedExtensions();
+  }
+
+  private static String jsonErrorResponse(String errorStatus, String errorScope,
+    String errorOpenIDConfiguration) {
+    String jsonErrorResponse = String.format("{\"status\":\"%s\"", errorStatus);
+    if (errorScope != null) {
+      jsonErrorResponse = String.format("%s, \"scope\":\"%s\"", jsonErrorResponse, errorScope);
+    }
+    if (errorOpenIDConfiguration != null) {
+      jsonErrorResponse = String.format("%s, \"openid-configuration\":\"%s\"", jsonErrorResponse,
+        errorOpenIDConfiguration);
+    }
+    jsonErrorResponse = String.format("%s}", jsonErrorResponse);
+    return jsonErrorResponse;

Review comment:
       What do you recommend here?
   I don't we need full-blown json generating here, because the variables are populated by us, they don't contain user input.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779714133



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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 javax.security.sasl.SaslServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This exception indicates that SASL authentication has failed. The error message
+ * in the exception indicates the actual cause of failure.
+ * <p>
+ * SASL authentication failures typically indicate invalid credentials, but
+ * could also include other failures specific to the SASL mechanism used
+ * for authentication.
+ * </p>
+ * <p><b>Note:</b>If {@link SaslServer#evaluateResponse(byte[])} throws this exception during
+ * authentication, the message from the exception will be sent to clients in the SaslAuthenticate
+ * response. Custom {@link SaslServer} implementations may throw this exception in order to
+ * provide custom error messages to clients, but should take care not to include any
+ * security-critical information in the message that should not be leaked to unauthenticated
+ * clients.
+ * </p>
+ */
+@InterfaceAudience.Public
+public class SaslAuthenticationException extends RuntimeException {

Review comment:
       Ok, I missed that. That's a good reason. We may have to do some unwrapping later (SaslAuthenticationException -> ... -> AccessDeniedException), but we're historically not great at throwing a clear exception :)




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779629490



##########
File path: hbase-common/pom.xml
##########
@@ -255,6 +255,10 @@
       <artifactId>kerb-simplekdc</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.nimbusds</groupId>
+      <artifactId>nimbus-jose-jwt</artifactId>

Review comment:
       I need to look into this, because currently most of the code is in hbase-common and I can't remember why. I'll try to move this to the server.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1005422265


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 21s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m  7s |  master passed  |
   | -0 :warning: |  patch  |  14m  0s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 19s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  2s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  2s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-common generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  javadoc  |   2m 18s |  root generated 1 new + 57 unchanged - 0 fixed = 58 total (was 57)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 427m 24s |  root in the patch failed.  |
   |  |   | 473m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux be5b43feb684 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 70cb9b0ba9 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/testReport/ |
   | Max. process+thread count | 4502 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778087673



##########
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:
       Refactored. Thanks!




-- 
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



[GitHub] [hbase] anmolnar closed pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
anmolnar closed pull request #3934:
URL: https://github.com/apache/hbase/pull/3934


   


-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1005336497


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 25s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 12s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 26s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 57s |  master passed  |
   | -0 :warning: |  patch  |  14m 26s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 16s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 16s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m  0s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 194m 42s |  root in the patch passed.  |
   |  |   | 242m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux ae9d3a0d81b7 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 70cb9b0ba9 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/testReport/ |
   | Max. process+thread count | 6963 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779637060



##########
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 javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+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 3.0.0 and, while it feels stable, it could
+ * evolve. We will try to evolve the API in a compatible manner, but we reserve
+ * the right to make breaking changes in minor releases, if necessary. We will
+ * update the {@code InterfaceStability} annotation and this notice once the API
+ * is considered stable.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerTokenCallback implements Callback {
+  private OAuthBearerToken token = null;
+  private String errorCode = null;
+  private String errorDescription = null;
+  private String errorUri = null;
+
+  /**
+   * Return the (potentially null) token
+   *
+   * @return the (potentially null) token
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * Return the optional (but always non-empty if not null) error code as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the optional (but always non-empty if not null) error code
+   */
+  public String errorCode() {
+    return errorCode;
+  }
+
+  /**
+   * Return the (potentially null) error description as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error description
+   */
+  public String errorDescription() {
+    return errorDescription;
+  }
+
+  /**
+   * Return the (potentially null) error URI as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error URI
+   */
+  public String errorUri() {
+    return errorUri;
+  }
+
+  /**
+   * Set the token. All error-related values are cleared.
+   *
+   * @param token
+   *            the optional token to set
+   */
+  public void token(OAuthBearerToken token) {

Review comment:
       I'm not sure how to do that. The callback instantiated by the SaslClient and token is populated by the callback handler which is further down the call chain.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778106762



##########
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:
       Right. `hbase-site.xml` must be set for the server side, it should have the server side providers already.
   I can add the client one here though.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1007010885


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 13s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 47s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 16s |  master passed  |
   | -0 :warning: |  patch  |  13m 25s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  6s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 44s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 44s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 25s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-common generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  javadoc  |   2m 13s |  root generated 1 new + 57 unchanged - 0 fixed = 58 total (was 57)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 382m 56s |  root in the patch failed.  |
   |  |   | 426m 34s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 4df5c30cde57 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7972b2ebbb |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/testReport/ |
   | Max. process+thread count | 5178 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778093873



##########
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:
       I already refactored these tests to use the new java.time api, so it should be already fine.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779574662



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.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.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 {

Review comment:
       It's thrown [here](https://github.com/apache/hbase/pull/3934/files#diff-69b635edb6eb466305c5aaeb515bae8634c06952b235681d67b0b5e4b7b454dcR120).
   The overriden method must throw `SaslException`, so we don't have to many options.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1006895892


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 56s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 18s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 32s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m 24s |  master passed  |
   | -0 :warning: |  patch  |  14m 43s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 36s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 16s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 16s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 50s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 172m 14s |  root in the patch failed.  |
   |  |   | 221m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux cc204cba9dc5 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7972b2ebbb |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/testReport/ |
   | Max. process+thread count | 3785 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779570699



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.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.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 {

Review comment:
       I see 2 problems with that:
   1. `DoNotRetryIOException` is in hbase-client module and cannot be used in hbase-common
   2. It's not a RuntimeException so cannot be replaced easily.
   
   Though this exception is only used at one place in the PR, so I'm happy with any suggestion to get rid of that.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-990964430


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 57s |  master passed  |
   | +1 :green_heart: |  compile  |  10m 57s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 26s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  19m  1s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 36s |  the patch passed  |
   | +1 :green_heart: |  compile  |  11m  4s |  the patch passed  |
   | -0 :warning: |  javac  |  11m  4s |  root generated 34 new + 1632 unchanged - 0 fixed = 1666 total (was 1632)  |
   | -0 :warning: |  checkstyle  |   2m 26s |  root: The patch generated 12 new + 17 unchanged - 0 fixed = 29 total (was 17)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  4s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  23m 50s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |  20m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 17s |  The patch does not generate ASF License warnings.  |
   |  |   | 114m 26s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 4a0ed75e831f 4.15.0-163-generic #171-Ubuntu SMP Fri Nov 5 11:55:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / b5cf3cdc5c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | Max. process+thread count | 126 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
meszibalu commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r773039936



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.OAUTHBEARER_MECHANISM;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.AccessController;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.SaslUtil;
+import org.apache.hadoop.hbase.security.SecurityInfo;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.hadoop.hbase.security.auth.SaslExtensionsCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
+
+@InterfaceAudience.Private
+public class OAuthBearerSaslClientAuthenticationProvider
+    extends OAuthBearerSaslAuthenticationProvider
+    implements SaslClientAuthenticationProvider {
+
+  @Override
+  public SaslClient createClient(Configuration conf, InetAddress serverAddr,
+                                 SecurityInfo securityInfo, Token<? extends TokenIdentifier> token,
+                                 boolean fallbackAllowed,
+                                 Map<String, String> saslProps) throws IOException {
+    AuthenticateCallbackHandler callbackHandler = new OAuthBearerSaslClientCallbackHandler();
+    callbackHandler.configure(conf, getSaslAuthMethod().getSaslMechanism(), saslProps);
+    return Sasl.createSaslClient(new String[] { getSaslAuthMethod().getSaslMechanism() }, null,
+        null, SaslUtil.SASL_DEFAULT_REALM, saslProps, callbackHandler);
+  }
+
+  public static class OAuthBearerSaslClientCallbackHandler implements AuthenticateCallbackHandler {
+    private static final Logger LOG =
+      LoggerFactory.getLogger(OAuthBearerSaslClientCallbackHandler.class);
+    private boolean configured = false;
+
+    @Override public void configure(Configuration configs, String saslMechanism,
+      Map<String, String> saslProps) {
+      if (!OAUTHBEARER_MECHANISM.equals(saslMechanism)) {
+        throw new IllegalArgumentException(
+          String.format("Unexpected SASL mechanism: %s", saslMechanism));
+      }
+      this.configured = true;
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+      if (!configured) {
+        throw new RuntimeException(

Review comment:
       IllegalStateException would be better.

##########
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
+ * evolve. We will try to evolve the API in a compatible manner, but we reserve
+ * the right to make breaking changes in minor releases, if necessary. We will
+ * update the {@code InterfaceStability} annotation and this notice once the API
+ * is considered stable.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerTokenCallback implements Callback {
+  private OAuthBearerToken token = null;
+  private String errorCode = null;
+  private String errorDescription = null;
+  private String errorUri = null;
+
+  /**
+   * Return the (potentially null) token
+   *
+   * @return the (potentially null) token
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * Return the optional (but always non-empty if not null) error code as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the optional (but always non-empty if not null) error code
+   */
+  public String errorCode() {
+    return errorCode;
+  }
+
+  /**
+   * Return the (potentially null) error description as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error description
+   */
+  public String errorDescription() {
+    return errorDescription;
+  }
+
+  /**
+   * Return the (potentially null) error URI as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error URI
+   */
+  public String errorUri() {
+    return errorUri;
+  }
+
+  /**
+   * Set the token. All error-related values are cleared.
+   *
+   * @param token
+   *            the optional token to set
+   */
+  public void token(OAuthBearerToken token) {
+    this.token = token;
+    this.errorCode = null;
+    this.errorDescription = null;
+    this.errorUri = null;
+  }
+
+  /**
+   * Set the error values as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>. Any token is cleared.
+   *
+   * @param errorCode
+   *            the mandatory error code to set
+   * @param errorDescription
+   *            the optional error description to set
+   * @param errorUri
+   *            the optional error URI to set
+   */
+  public void error(String errorCode, String errorDescription, String errorUri) {
+    if (Objects.requireNonNull(errorCode).isEmpty()) {

Review comment:
       Same here as above.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerIllegalTokenException.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.internals.knox;
+
+import java.util.Objects;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Exception thrown when token validation fails due to a problem with the token
+ * itself (as opposed to a missing remote resource or a configuration problem)
+ */
+@InterfaceAudience.Public
+public class OAuthBearerIllegalTokenException extends RuntimeException {
+  private static final long serialVersionUID = -5275276640051316350L;
+  private final OAuthBearerValidationResult reason;
+
+  /**
+   * Constructor
+   *
+   * @param reason
+   *            the mandatory reason for the validation failure; it must indicate
+   *            failure
+   */
+  public OAuthBearerIllegalTokenException(OAuthBearerValidationResult reason) {
+    super(Objects.requireNonNull(reason).failureDescription());
+    if (reason.success()) {
+      throw new IllegalArgumentException(
+        "The reason indicates success; it must instead indicate failure");
+    }
+    this.reason = reason;
+  }
+
+  public OAuthBearerIllegalTokenException(OAuthBearerValidationResult reason, Throwable t) {
+    super(Objects.requireNonNull(reason).failureDescription(), t);

Review comment:
       Same here.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslClientAuthenticationProvider.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.apache.hadoop.hbase.security.token.OAuthBearerTokenUtil.OAUTHBEARER_MECHANISM;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.AccessController;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.SaslUtil;
+import org.apache.hadoop.hbase.security.SecurityInfo;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.hadoop.hbase.security.auth.SaslExtensionsCallback;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerTokenCallback;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
+
+@InterfaceAudience.Private
+public class OAuthBearerSaslClientAuthenticationProvider
+    extends OAuthBearerSaslAuthenticationProvider
+    implements SaslClientAuthenticationProvider {
+
+  @Override
+  public SaslClient createClient(Configuration conf, InetAddress serverAddr,
+                                 SecurityInfo securityInfo, Token<? extends TokenIdentifier> token,
+                                 boolean fallbackAllowed,
+                                 Map<String, String> saslProps) throws IOException {
+    AuthenticateCallbackHandler callbackHandler = new OAuthBearerSaslClientCallbackHandler();
+    callbackHandler.configure(conf, getSaslAuthMethod().getSaslMechanism(), saslProps);
+    return Sasl.createSaslClient(new String[] { getSaslAuthMethod().getSaslMechanism() }, null,
+        null, SaslUtil.SASL_DEFAULT_REALM, saslProps, callbackHandler);
+  }
+
+  public static class OAuthBearerSaslClientCallbackHandler implements AuthenticateCallbackHandler {
+    private static final Logger LOG =
+      LoggerFactory.getLogger(OAuthBearerSaslClientCallbackHandler.class);
+    private boolean configured = false;
+
+    @Override public void configure(Configuration configs, String saslMechanism,
+      Map<String, String> saslProps) {
+      if (!OAUTHBEARER_MECHANISM.equals(saslMechanism)) {
+        throw new IllegalArgumentException(
+          String.format("Unexpected SASL mechanism: %s", saslMechanism));
+      }
+      this.configured = true;
+    }
+
+    @Override
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+      if (!configured) {
+        throw new RuntimeException(
+          "OAuthBearerSaslClientCallbackHandler handler must be configured first.");
+      }
+
+      for (Callback callback : callbacks) {
+        if (callback instanceof OAuthBearerTokenCallback) {
+          handleCallback((OAuthBearerTokenCallback) callback);
+        } else if (callback instanceof SaslExtensionsCallback) {
+          handleCallback((SaslExtensionsCallback) callback,
+            Subject.getSubject(AccessController.getContext()));
+        } else {
+          throw new UnsupportedCallbackException(callback);
+        }
+      }
+    }
+
+    private void handleCallback(OAuthBearerTokenCallback callback) throws IOException {
+      if (callback.token() != null) {
+        throw new IllegalArgumentException("Callback had a token already");
+      }
+      Subject subject = Subject.getSubject(AccessController.getContext());
+      Set<OAuthBearerToken> privateCredentials = subject != null
+        ? subject.getPrivateCredentials(OAuthBearerToken.class)
+        : Collections.emptySet();
+      if (privateCredentials.size() == 0) {
+        throw new IOException("No OAuth Bearer tokens in Subject's private credentials");
+      }
+      if (privateCredentials.size() == 1) {
+        LOG.debug("Found 1 OAuthBearer token");
+        callback.token(privateCredentials.iterator().next());
+      } else {
+        /*
+         * There a very small window of time upon token refresh (on the order of milliseconds)
+         * where both an old and a new token appear on the Subject's private credentials.
+         * Rather than implement a lock to eliminate this window, we will deal with it by
+         * checking for the existence of multiple tokens and choosing the one that has the
+         * longest lifetime.  It is also possible that a bug could cause multiple tokens to
+         * exist (e.g. KAFKA-7902), so dealing with the unlikely possibility that occurs
+         * during normal operation also allows us to deal more robustly with potential bugs.
+         */
+        SortedSet<OAuthBearerToken> sortedByLifetime =
+          new TreeSet<>(
+            new Comparator<OAuthBearerToken>() {
+              @Override
+              public int compare(OAuthBearerToken o1, OAuthBearerToken o2) {
+                return Long.compare(o1.lifetimeMs(), o2.lifetimeMs());
+              }
+            });
+        sortedByLifetime.addAll(privateCredentials);
+        LOG.warn("Found {} OAuth Bearer tokens in Subject's private credentials; "
+            + "the oldest expires at {}, will use the newest, which expires at {}",
+          sortedByLifetime.size(),
+          new Date(sortedByLifetime.first().lifetimeMs()),
+          new Date(sortedByLifetime.last().lifetimeMs()));

Review comment:
       If the log message creation consist of multiple steps, please add `if (LOG.isWarnEnabled())`.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.Utils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own {@link OAuthBearerValidatorCallback}
+ * irregardless of provided extensions, as they are inherently insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * @return {@link SaslExtensions} consisting of the unvalidated extension names and values that
+   * were sent by the client
+   */
+  public SaslExtensions inputExtensions() {
+    return inputExtensions;
+  }
+
+  /**
+   * @return an unmodifiable {@link Map} consisting of the validated and recognized by the server
+   * extension names and values.
+   */
+  public Map<String, String> validatedExtensions() {
+    return Collections.unmodifiableMap(validatedExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the name-&gt;error messages of extensions
+   * which failed validation
+   */
+  public Map<String, String> invalidExtensions() {
+    return Collections.unmodifiableMap(invalidExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the extensions that have neither been
+   * validated nor invalidated
+   */
+  public Map<String, String> ignoredExtensions() {
+    return Collections.unmodifiableMap(
+      subtractMap(subtractMap(inputExtensions.map(), invalidExtensions), validatedExtensions));
+  }
+
+  /**
+   * Validates a specific extension in the original {@code inputExtensions} map
+   * @param extensionName - the name of the extension which was validated
+   */
+  public void valid(String extensionName) {
+    if (!inputExtensions.map().containsKey(extensionName)) {
+      throw new IllegalArgumentException(
+        String.format("Extension %s was not found in the original extensions", extensionName));
+    }
+    validatedExtensions.put(extensionName, inputExtensions.map().get(extensionName));
+  }
+  /**
+   * Set the error value for a specific extension key-value pair if validation has failed
+   *
+   * @param invalidExtensionName
+   *            the mandatory extension name which caused the validation failure
+   * @param errorMessage
+   *            error message describing why the validation failed
+   */
+  public void error(String invalidExtensionName, String errorMessage) {
+    if (Objects.requireNonNull(invalidExtensionName).isEmpty()) {

Review comment:
       1. You can use `Strings.isEmpty` from guava.
   2. Or `StringUtils.isEmpty` from commons-lang.
   3. You didn't pass a string for `requireNotNull`, so it will throw the same undescriptive NPE as `invalidExtensionName.isEmpty()`.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallback.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 SaslServer} implementation when it
+ * needs to provide an OAuth 2 bearer token compact serialization for
+ * validation. Callback handlers should use the
+ * {@link #error(String, String, String)} method to communicate errors back to
+ * the SASL Client as per
+ * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+ * 2.0 Authorization Framework</a> and the <a href=
+ * "https://www.iana.org/assignments/oauth-parameters/oauth-parameters.xhtml#extensions-error">IANA
+ * OAuth Extensions Error Registry</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
+ * evolve. We will try to evolve the API in a compatible manner, but we reserve
+ * the right to make breaking changes in minor releases, if necessary. We will
+ * update the {@code InterfaceStability} annotation and this notice once the API
+ * is considered stable.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerValidatorCallback implements Callback {
+  private final String tokenValue;
+  private OAuthBearerToken token = null;
+  private String errorStatus = null;
+  private String errorScope = null;
+  private String errorOpenIDConfiguration = null;
+
+  /**
+   * Constructor
+   *
+   * @param tokenValue
+   *            the mandatory/non-blank token value
+   */
+  public OAuthBearerValidatorCallback(String tokenValue) {
+    if (Objects.requireNonNull(tokenValue).isEmpty()) {

Review comment:
       And here.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerIllegalTokenException.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.internals.knox;
+
+import java.util.Objects;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Exception thrown when token validation fails due to a problem with the token
+ * itself (as opposed to a missing remote resource or a configuration problem)
+ */
+@InterfaceAudience.Public
+public class OAuthBearerIllegalTokenException extends RuntimeException {
+  private static final long serialVersionUID = -5275276640051316350L;
+  private final OAuthBearerValidationResult reason;
+
+  /**
+   * Constructor
+   *
+   * @param reason
+   *            the mandatory reason for the validation failure; it must indicate
+   *            failure
+   */
+  public OAuthBearerIllegalTokenException(OAuthBearerValidationResult reason) {
+    super(Objects.requireNonNull(reason).failureDescription());

Review comment:
       `super(reason.failureDescription())` would throw the same undescriptive NPE as this. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java
##########
@@ -0,0 +1,99 @@
+/*
+ *  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 java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.OAuthBearerSaslServerProvider;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.knox.OAuthBearerSignedJwtValidatorCallbackHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class OAuthBearerSaslServerAuthenticationProvider
+    extends OAuthBearerSaslAuthenticationProvider
+    implements SaslServerAuthenticationProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    OAuthBearerSaslServerAuthenticationProvider.class);
+  private Configuration hbaseConfiguration;
+  private boolean initialized = false;
+
+  static {
+    OAuthBearerSaslServerProvider.initialize(); // not part of public API
+    LOG.info("OAuthBearer SASL server provider has been initialized");
+  }
+
+  @Override public void init(Configuration conf) throws IOException {
+    this.hbaseConfiguration = conf;
+    this.initialized = true;
+  }
+
+  @Override public AttemptingUserProvidingSaslServer createServer(
+    SecretManager<TokenIdentifier> secretManager, Map<String, String> saslProps)
+    throws IOException {
+
+    if (!initialized) {
+      throw new RuntimeException(

Review comment:
       What about throwing an `IllegalStateException`?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerValidatorCallback.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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 SaslServer} implementation when it
+ * needs to provide an OAuth 2 bearer token compact serialization for
+ * validation. Callback handlers should use the
+ * {@link #error(String, String, String)} method to communicate errors back to
+ * the SASL Client as per
+ * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+ * 2.0 Authorization Framework</a> and the <a href=
+ * "https://www.iana.org/assignments/oauth-parameters/oauth-parameters.xhtml#extensions-error">IANA
+ * OAuth Extensions Error Registry</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
+ * evolve. We will try to evolve the API in a compatible manner, but we reserve
+ * the right to make breaking changes in minor releases, if necessary. We will
+ * update the {@code InterfaceStability} annotation and this notice once the API
+ * is considered stable.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerValidatorCallback implements Callback {
+  private final String tokenValue;
+  private OAuthBearerToken token = null;
+  private String errorStatus = null;
+  private String errorScope = null;
+  private String errorOpenIDConfiguration = null;
+
+  /**
+   * Constructor
+   *
+   * @param tokenValue
+   *            the mandatory/non-blank token value
+   */
+  public OAuthBearerValidatorCallback(String tokenValue) {
+    if (Objects.requireNonNull(tokenValue).isEmpty()) {
+      throw new IllegalArgumentException("token value must not be empty");
+    }
+    this.tokenValue = tokenValue;
+  }
+
+  /**
+   * Return the (always non-null) token value
+   *
+   * @return the (always non-null) token value
+   */
+  public String tokenValue() {
+    return tokenValue;
+  }
+
+  /**
+   * Return the (potentially null) token
+   *
+   * @return the (potentially null) token
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * Return the (potentially null) error status value as per
+   * <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
+   * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>
+   * and the <a href=
+   * "https://www.iana.org/assignments/oauth-parameters/oauth-parameters.xhtml#extensions-error">IANA
+   * OAuth Extensions Error Registry</a>.
+   *
+   * @return the (potentially null) error status value
+   */
+  public String errorStatus() {
+    return errorStatus;
+  }
+
+  /**
+   * Return the (potentially null) error scope value as per
+   * <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
+   * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>.
+   *
+   * @return the (potentially null) error scope value
+   */
+  public String errorScope() {
+    return errorScope;
+  }
+
+  /**
+   * Return the (potentially null) error openid-configuration value as per
+   * <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
+   * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>.
+   *
+   * @return the (potentially null) error openid-configuration value
+   */
+  public String errorOpenIDConfiguration() {
+    return errorOpenIDConfiguration;
+  }
+
+  /**
+   * Set the token. The token value is unchanged and is expected to match the
+   * provided token's value. All error values are cleared.
+   *
+   * @param token
+   *            the mandatory token to set
+   */
+  public void token(OAuthBearerToken token) {
+    this.token = Objects.requireNonNull(token);
+    this.errorStatus = null;
+    this.errorScope = null;
+    this.errorOpenIDConfiguration = null;
+  }
+
+  /**
+   * Set the error values as per
+   * <a href="https://tools.ietf.org/html/rfc7628#section-3.2.2">RFC 7628: A Set
+   * of Simple Authentication and Security Layer (SASL) Mechanisms for OAuth</a>.
+   * Any token is cleared.
+   *
+   * @param errorStatus
+   *            the mandatory error status value from the <a href=
+   *            "https://www.iana.org/assignments/oauth-parameters/oauth-parameters.xhtml#extensions-error">IANA
+   *            OAuth Extensions Error Registry</a> to set
+   * @param errorScope
+   *            the optional error scope value to set
+   * @param errorOpenIDConfiguration
+   *            the optional error openid-configuration value to set
+   */
+  public void error(String errorStatus, String errorScope, String errorOpenIDConfiguration) {
+    if (Objects.requireNonNull(errorStatus).isEmpty()) {

Review comment:
       And here.

##########
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)) {

Review comment:
       `instanceof` returns `false` for null, so no need for `requireNonNull`.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java
##########
@@ -0,0 +1,99 @@
+/*
+ *  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 java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.OAuthBearerSaslServerProvider;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.knox.OAuthBearerSignedJwtValidatorCallbackHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class OAuthBearerSaslServerAuthenticationProvider
+    extends OAuthBearerSaslAuthenticationProvider
+    implements SaslServerAuthenticationProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    OAuthBearerSaslServerAuthenticationProvider.class);
+  private Configuration hbaseConfiguration;
+  private boolean initialized = false;
+
+  static {
+    OAuthBearerSaslServerProvider.initialize(); // not part of public API
+    LOG.info("OAuthBearer SASL server provider has been initialized");
+  }
+
+  @Override public void init(Configuration conf) throws IOException {
+    this.hbaseConfiguration = conf;
+    this.initialized = true;
+  }
+
+  @Override public AttemptingUserProvidingSaslServer createServer(
+    SecretManager<TokenIdentifier> secretManager, Map<String, String> saslProps)
+    throws IOException {
+
+    if (!initialized) {
+      throw new RuntimeException(
+        "OAuthBearerSaslServerAuthenticationProvider must be initialized first.");
+    }
+
+    UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    String fullName = current.getUserName();
+    LOG.debug("Server's OAuthBearer user name is {}", fullName);
+    LOG.debug("OAuthBearer saslProps = {}", saslProps);
+
+    try {
+      return current.doAs(new PrivilegedExceptionAction<AttemptingUserProvidingSaslServer>() {
+        @Override
+        public AttemptingUserProvidingSaslServer run() throws SaslException {
+          AuthenticateCallbackHandler callbackHandler =
+            new OAuthBearerSignedJwtValidatorCallbackHandler();
+          callbackHandler.configure(hbaseConfiguration, getSaslAuthMethod().getSaslMechanism(),
+            saslProps);
+          return new AttemptingUserProvidingSaslServer(Sasl.createSaslServer(
+            getSaslAuthMethod().getSaslMechanism(), null, null, saslProps,
+            callbackHandler), () -> null);
+        }
+      });
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Failed to construct OAUTHBEARER SASL server");

Review comment:
       Please use another exception type.

##########
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());
+      throw e;
+    } catch (Exception e) {
+      LOG.error("SASL server problem", e);
+      throw e;
+    }
+  }
+
+  @Override
+  public String getAuthorizationID() {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return tokenForNegotiatedProperty.principalName();
+  }
+
+  @Override
+  public String getMechanismName() {
+    return OAUTHBEARER_MECHANISM;
+  }
+
+  @Override
+  public Object getNegotiatedProperty(String propName) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    if (NEGOTIATED_PROPERTY_KEY_TOKEN.equals(propName)) {
+      return tokenForNegotiatedProperty;
+    }
+    if (CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY.equals(propName)) {
+      return tokenForNegotiatedProperty.lifetimeMs();
+    }
+    return extensions.map().get(propName);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return complete;
+  }
+
+  @Override
+  public byte[] unwrap(byte[] incoming, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(incoming, offset, offset + len);
+  }
+
+  @Override
+  public byte[] wrap(byte[] outgoing, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(outgoing, offset, offset + len);
+  }
+
+  @Override
+  public void dispose() {
+    complete = false;
+    tokenForNegotiatedProperty = null;
+    extensions = null;
+  }
+
+  private byte[] process(String tokenValue, String authorizationId, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(tokenValue);
+    try {
+      callbackHandler.handle(new Callback[] {callback});
+    } catch (IOException | UnsupportedCallbackException e) {
+      handleCallbackError(e);
+    }
+    OAuthBearerToken token = callback.token();
+    if (token == null) {
+      errorMessage = jsonErrorResponse(callback.errorStatus(), callback.errorScope(),
+        callback.errorOpenIDConfiguration());
+      LOG.error("JWT token validation error: {}", errorMessage);
+      return errorMessage.getBytes(StandardCharsets.UTF_8);
+    }
+    /*
+     * We support the client specifying an authorization ID as per the SASL
+     * specification, but it must match the principal name if it is specified.
+     */
+    if (!authorizationId.isEmpty() && !authorizationId.equals(token.principalName())) {
+      throw new SaslAuthenticationException(String.format(
+        "Authentication failed: Client requested an authorization id (%s) that is different from "
+          + "the token's principal name (%s)",
+        authorizationId, token.principalName()));
+    }
+
+    Map<String, String> validExtensions = processExtensions(token, extensions);
+
+    tokenForNegotiatedProperty = token;
+    this.extensions = new SaslExtensions(validExtensions);
+    complete = true;
+    LOG.debug("Successfully authenticate User={}", token.principalName());
+    return new byte[0];
+  }
+
+  private Map<String, String> processExtensions(OAuthBearerToken token, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerExtensionsValidatorCallback
+      extensionsCallback = new OAuthBearerExtensionsValidatorCallback(token, extensions);
+    try {
+      callbackHandler.handle(new Callback[] {extensionsCallback});
+    } catch (UnsupportedCallbackException e) {
+      // backwards compatibility - no extensions will be added
+    } catch (IOException e) {
+      handleCallbackError(e);
+    }
+    if (!extensionsCallback.invalidExtensions().isEmpty()) {
+      String errorMessage = String.format("Authentication failed: %d extensions are invalid! "
+          + "They are: %s", extensionsCallback.invalidExtensions().size(),
+        Utils.mkString(extensionsCallback.invalidExtensions(), "", "", ": ", "; "));
+      LOG.debug(errorMessage);
+      throw new SaslAuthenticationException(errorMessage);
+    }
+
+    return extensionsCallback.validatedExtensions();
+  }
+
+  private static String jsonErrorResponse(String errorStatus, String errorScope,
+    String errorOpenIDConfiguration) {
+    String jsonErrorResponse = String.format("{\"status\":\"%s\"", errorStatus);
+    if (errorScope != null) {
+      jsonErrorResponse = String.format("%s, \"scope\":\"%s\"", jsonErrorResponse, errorScope);
+    }
+    if (errorOpenIDConfiguration != null) {
+      jsonErrorResponse = String.format("%s, \"openid-configuration\":\"%s\"", jsonErrorResponse,
+        errorOpenIDConfiguration);
+    }
+    jsonErrorResponse = String.format("%s}", jsonErrorResponse);
+    return jsonErrorResponse;

Review comment:
       `String.format` is not a good approach to create a JSON. You should take care of the escaping at least.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779716079



##########
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 javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+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 3.0.0 and, while it feels stable, it could
+ * evolve. We will try to evolve the API in a compatible manner, but we reserve
+ * the right to make breaking changes in minor releases, if necessary. We will
+ * update the {@code InterfaceStability} annotation and this notice once the API
+ * is considered stable.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerTokenCallback implements Callback {
+  private OAuthBearerToken token = null;
+  private String errorCode = null;
+  private String errorDescription = null;
+  private String errorUri = null;
+
+  /**
+   * Return the (potentially null) token
+   *
+   * @return the (potentially null) token
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * Return the optional (but always non-empty if not null) error code as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the optional (but always non-empty if not null) error code
+   */
+  public String errorCode() {
+    return errorCode;
+  }
+
+  /**
+   * Return the (potentially null) error description as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error description
+   */
+  public String errorDescription() {
+    return errorDescription;
+  }
+
+  /**
+   * Return the (potentially null) error URI as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error URI
+   */
+  public String errorUri() {
+    return errorUri;
+  }
+
+  /**
+   * Set the token. All error-related values are cleared.
+   *
+   * @param token
+   *            the optional token to set
+   */
+  public void token(OAuthBearerToken token) {

Review comment:
       Ah ok.  That's a good reason for it to be mutable.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779097090



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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) {

Review comment:
       I think it might be better to fail hard (throw exception) if we try to construct the SaslServer in an inherently insecure manner. Then we could get rid of all of the indirection in this class about compatibility with mechanism/policy.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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 (!(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);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("SASL server problem", e);
+      throw e;
+    }
+  }
+
+  @Override
+  public String getAuthorizationID() {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return tokenForNegotiatedProperty.principalName();
+  }
+
+  @Override
+  public String getMechanismName() {
+    return OAUTHBEARER_MECHANISM;
+  }
+
+  @Override
+  public Object getNegotiatedProperty(String propName) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    if (NEGOTIATED_PROPERTY_KEY_TOKEN.equals(propName)) {
+      return tokenForNegotiatedProperty;
+    }
+    if (CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY.equals(propName)) {
+      return tokenForNegotiatedProperty.lifetimeMs();
+    }
+    return extensions.map().get(propName);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return complete;
+  }
+
+  @Override
+  public byte[] unwrap(byte[] incoming, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(incoming, offset, offset + len);
+  }
+
+  @Override
+  public byte[] wrap(byte[] outgoing, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(outgoing, offset, offset + len);
+  }
+
+  @Override
+  public void dispose() {
+    complete = false;
+    tokenForNegotiatedProperty = null;
+    extensions = null;
+  }
+
+  private byte[] process(String tokenValue, String authorizationId, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(tokenValue);
+    try {
+      callbackHandler.handle(new Callback[] {callback});
+    } catch (IOException | UnsupportedCallbackException e) {
+      handleCallbackError(e);
+    }
+    OAuthBearerToken token = callback.token();
+    if (token == null) {
+      errorMessage = jsonErrorResponse(callback.errorStatus(), callback.errorScope(),
+        callback.errorOpenIDConfiguration());
+      LOG.error("JWT token validation error: {}", errorMessage);
+      return errorMessage.getBytes(StandardCharsets.UTF_8);
+    }
+    /*
+     * We support the client specifying an authorization ID as per the SASL
+     * specification, but it must match the principal name if it is specified.
+     */
+    if (!authorizationId.isEmpty() && !authorizationId.equals(token.principalName())) {
+      throw new SaslAuthenticationException(String.format(
+        "Authentication failed: Client requested an authorization id (%s) that is different from "
+          + "the token's principal name (%s)",
+        authorizationId, token.principalName()));
+    }
+
+    Map<String, String> validExtensions = processExtensions(token, extensions);
+
+    tokenForNegotiatedProperty = token;
+    this.extensions = new SaslExtensions(validExtensions);
+    complete = true;
+    LOG.debug("Successfully authenticate User={}", token.principalName());
+    return new byte[0];
+  }
+
+  private Map<String, String> processExtensions(OAuthBearerToken token, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerExtensionsValidatorCallback
+      extensionsCallback = new OAuthBearerExtensionsValidatorCallback(token, extensions);
+    try {
+      callbackHandler.handle(new Callback[] {extensionsCallback});
+    } catch (UnsupportedCallbackException e) {
+      // backwards compatibility - no extensions will be added
+    } catch (IOException e) {
+      handleCallbackError(e);
+    }
+    if (!extensionsCallback.invalidExtensions().isEmpty()) {
+      String errorMessage = String.format("Authentication failed: %d extensions are invalid! "
+          + "They are: %s", extensionsCallback.invalidExtensions().size(),
+        OAuthBearerStringUtils.mkString(extensionsCallback.invalidExtensions(),
+          "", "", ": ", "; "));
+      LOG.debug(errorMessage);
+      throw new SaslAuthenticationException(errorMessage);
+    }
+
+    return extensionsCallback.validatedExtensions();
+  }
+
+  private static String jsonErrorResponse(String errorStatus, String errorScope,
+    String errorOpenIDConfiguration) {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put("status", errorStatus);
+    if (!StringUtils.isBlank(errorScope)) {
+      jsonObject.put("scope", errorScope);
+    }
+    if (!StringUtils.isBlank(errorOpenIDConfiguration)) {
+      jsonObject.put("openid-configuration", errorOpenIDConfiguration);
+    }
+    return jsonObject.toJSONString();
+  }
+
+  private void handleCallbackError(Exception e) throws SaslException {
+    String msg = String.format("%s: %s", INTERNAL_ERROR_ON_SERVER, e.getMessage());
+    LOG.debug(msg, e);
+    throw new SaslException(msg);
+  }
+
+  public static String[] mechanismNamesCompatibleWithPolicy(Map<String, ?> props) {
+    return props != null && "true".equals(String.valueOf(props.get(Sasl.POLICY_NOPLAINTEXT)))

Review comment:
       `Boolean.valueOf(String.valueOf(props.get(Sasl.POLICY_NOPLAINTEXT)))` is a little more succient.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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 (!(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);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("SASL server problem", e);
+      throw e;
+    }
+  }
+
+  @Override
+  public String getAuthorizationID() {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return tokenForNegotiatedProperty.principalName();
+  }
+
+  @Override
+  public String getMechanismName() {
+    return OAUTHBEARER_MECHANISM;
+  }
+
+  @Override
+  public Object getNegotiatedProperty(String propName) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    if (NEGOTIATED_PROPERTY_KEY_TOKEN.equals(propName)) {
+      return tokenForNegotiatedProperty;
+    }
+    if (CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY.equals(propName)) {
+      return tokenForNegotiatedProperty.lifetimeMs();
+    }
+    return extensions.map().get(propName);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return complete;
+  }
+
+  @Override
+  public byte[] unwrap(byte[] incoming, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(incoming, offset, offset + len);
+  }
+
+  @Override
+  public byte[] wrap(byte[] outgoing, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(outgoing, offset, offset + len);
+  }
+
+  @Override
+  public void dispose() {
+    complete = false;
+    tokenForNegotiatedProperty = null;
+    extensions = null;
+  }
+
+  private byte[] process(String tokenValue, String authorizationId, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerValidatorCallback callback = new OAuthBearerValidatorCallback(tokenValue);
+    try {
+      callbackHandler.handle(new Callback[] {callback});
+    } catch (IOException | UnsupportedCallbackException e) {
+      handleCallbackError(e);
+    }
+    OAuthBearerToken token = callback.token();
+    if (token == null) {
+      errorMessage = jsonErrorResponse(callback.errorStatus(), callback.errorScope(),
+        callback.errorOpenIDConfiguration());
+      LOG.error("JWT token validation error: {}", errorMessage);
+      return errorMessage.getBytes(StandardCharsets.UTF_8);
+    }
+    /*
+     * We support the client specifying an authorization ID as per the SASL
+     * specification, but it must match the principal name if it is specified.
+     */
+    if (!authorizationId.isEmpty() && !authorizationId.equals(token.principalName())) {
+      throw new SaslAuthenticationException(String.format(
+        "Authentication failed: Client requested an authorization id (%s) that is different from "
+          + "the token's principal name (%s)",
+        authorizationId, token.principalName()));
+    }
+
+    Map<String, String> validExtensions = processExtensions(token, extensions);
+
+    tokenForNegotiatedProperty = token;
+    this.extensions = new SaslExtensions(validExtensions);
+    complete = true;
+    LOG.debug("Successfully authenticate User={}", token.principalName());
+    return new byte[0];
+  }
+
+  private Map<String, String> processExtensions(OAuthBearerToken token, SaslExtensions extensions)
+    throws SaslException {
+    OAuthBearerExtensionsValidatorCallback
+      extensionsCallback = new OAuthBearerExtensionsValidatorCallback(token, extensions);
+    try {
+      callbackHandler.handle(new Callback[] {extensionsCallback});
+    } catch (UnsupportedCallbackException e) {
+      // backwards compatibility - no extensions will be added
+    } catch (IOException e) {
+      handleCallbackError(e);
+    }
+    if (!extensionsCallback.invalidExtensions().isEmpty()) {
+      String errorMessage = String.format("Authentication failed: %d extensions are invalid! "
+          + "They are: %s", extensionsCallback.invalidExtensions().size(),
+        OAuthBearerStringUtils.mkString(extensionsCallback.invalidExtensions(),
+          "", "", ": ", "; "));
+      LOG.debug(errorMessage);
+      throw new SaslAuthenticationException(errorMessage);
+    }
+
+    return extensionsCallback.validatedExtensions();
+  }
+
+  private static String jsonErrorResponse(String errorStatus, String errorScope,
+    String errorOpenIDConfiguration) {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put("status", errorStatus);
+    if (!StringUtils.isBlank(errorScope)) {
+      jsonObject.put("scope", errorScope);
+    }
+    if (!StringUtils.isBlank(errorOpenIDConfiguration)) {
+      jsonObject.put("openid-configuration", errorOpenIDConfiguration);
+    }
+    return jsonObject.toJSONString();
+  }
+
+  private void handleCallbackError(Exception e) throws SaslException {
+    String msg = String.format("%s: %s", INTERNAL_ERROR_ON_SERVER, e.getMessage());
+    LOG.debug(msg, e);
+    throw new SaslException(msg);
+  }
+
+  public static String[] mechanismNamesCompatibleWithPolicy(Map<String, ?> props) {
+    return props != null && "true".equals(String.valueOf(props.get(Sasl.POLICY_NOPLAINTEXT)))

Review comment:
       I'm also tryign to make sure I understand this check -- if we disallow plaintext mechanism, we also disallow oauth bearer mechanism?
   
   In the context of HBase, would we ever want to allow the user to run HBase with oauth-bearer authentication without wire encryption? For a production scenario, I think the answer would be "no".

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java
##########
@@ -0,0 +1,99 @@
+/*
+ *  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 java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.OAuthBearerSaslServerProvider;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.knox.OAuthBearerSignedJwtValidatorCallbackHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class OAuthBearerSaslServerAuthenticationProvider
+    extends OAuthBearerSaslAuthenticationProvider
+    implements SaslServerAuthenticationProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    OAuthBearerSaslServerAuthenticationProvider.class);
+  private Configuration hbaseConfiguration;
+  private boolean initialized = false;
+
+  static {
+    OAuthBearerSaslServerProvider.initialize(); // not part of public API
+    LOG.info("OAuthBearer SASL server provider has been initialized");
+  }
+
+  @Override public void init(Configuration conf) throws IOException {
+    this.hbaseConfiguration = conf;
+    this.initialized = true;
+  }
+
+  @Override public AttemptingUserProvidingSaslServer createServer(
+    SecretManager<TokenIdentifier> secretManager, Map<String, String> saslProps)
+    throws IOException {
+
+    if (!initialized) {
+      throw new RuntimeException(
+        "OAuthBearerSaslServerAuthenticationProvider must be initialized first.");
+    }
+
+    UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    String fullName = current.getUserName();
+    LOG.debug("Server's OAuthBearer user name is {}", fullName);
+    LOG.debug("OAuthBearer saslProps = {}", saslProps);
+
+    try {
+      return current.doAs(new PrivilegedExceptionAction<AttemptingUserProvidingSaslServer>() {
+        @Override
+        public AttemptingUserProvidingSaslServer run() throws SaslException {
+          AuthenticateCallbackHandler callbackHandler =
+            new OAuthBearerSignedJwtValidatorCallbackHandler();
+          callbackHandler.configure(hbaseConfiguration, getSaslAuthMethod().getSaslMechanism(),
+            saslProps);
+          return new AttemptingUserProvidingSaslServer(Sasl.createSaslServer(
+            getSaslAuthMethod().getSaslMechanism(), null, null, saslProps,
+            callbackHandler), () -> null);
+        }
+      });
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Failed to construct OAUTHBEARER SASL server");

Review comment:
       In general, we use IOException as the base-class for caught exceptions. I'd suggest just throwing `InterruptedIOException`

##########
File path: hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java
##########
@@ -19,7 +19,6 @@
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-

Review comment:
       nit: In general, try to avoid making changes to unrelated files.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwt.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 com.nimbusds.jose.JOSEException;
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jose.jwk.JWKSet;
+import com.nimbusds.jose.jwk.source.ImmutableJWKSet;
+import com.nimbusds.jose.proc.BadJOSEException;
+import com.nimbusds.jose.proc.JWSKeySelector;
+import com.nimbusds.jose.proc.JWSVerificationKeySelector;
+import com.nimbusds.jose.proc.SecurityContext;
+import com.nimbusds.jwt.JWT;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.JWTParser;
+import com.nimbusds.jwt.proc.ConfigurableJWTProcessor;
+import com.nimbusds.jwt.proc.DefaultJWTClaimsVerifier;
+import com.nimbusds.jwt.proc.DefaultJWTProcessor;
+import java.text.ParseException;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Signed JWT implementation for OAuth Bearer authentication mech of SASL.
+ *
+ * This class is based on Kafka's Unsecured JWS token implementation.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerSignedJwt implements OAuthBearerToken {
+  private final String compactSerialization;
+  private final JWKSet jwkSet;
+
+  private JWTClaimsSet claims;
+  private long lifetime;
+  private int maxClockSkewSeconds = 0;

Review comment:
       Is the value of 0 disabling the clock skew validation?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/knox/OAuthBearerSignedJwt.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 com.nimbusds.jose.JOSEException;
+import com.nimbusds.jose.JWSAlgorithm;
+import com.nimbusds.jose.jwk.JWKSet;
+import com.nimbusds.jose.jwk.source.ImmutableJWKSet;
+import com.nimbusds.jose.proc.BadJOSEException;
+import com.nimbusds.jose.proc.JWSKeySelector;
+import com.nimbusds.jose.proc.JWSVerificationKeySelector;
+import com.nimbusds.jose.proc.SecurityContext;
+import com.nimbusds.jwt.JWT;
+import com.nimbusds.jwt.JWTClaimsSet;
+import com.nimbusds.jwt.JWTParser;
+import com.nimbusds.jwt.proc.ConfigurableJWTProcessor;
+import com.nimbusds.jwt.proc.DefaultJWTClaimsVerifier;
+import com.nimbusds.jwt.proc.DefaultJWTProcessor;
+import java.text.ParseException;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Signed JWT implementation for OAuth Bearer authentication mech of SASL.
+ *
+ * This class is based on Kafka's Unsecured JWS token implementation.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerSignedJwt implements OAuthBearerToken {
+  private final String compactSerialization;
+  private final JWKSet jwkSet;
+
+  private JWTClaimsSet claims;
+  private long lifetime;
+  private int maxClockSkewSeconds = 0;
+  private String requiredAudience;
+  private String requiredIssuer;
+
+  /**
+   * Constructor base64 encoded JWT token and JWK Set.
+   *
+   * @param compactSerialization
+   *            the compact serialization to parse as a signed JWT
+   * @param jwkSet
+   *            the key set which the signature of this JWT should be verified with
+   */
+  public OAuthBearerSignedJwt(String compactSerialization, JWKSet jwkSet) {

Review comment:
       Unless there's a reason, could we construct this via one constructor with all of the options or make a Builder class? (which could automatically validate when we construct the Token)

##########
File path: pom.xml
##########
@@ -1821,6 +1821,7 @@
     <xz.version>1.9</xz.version>
     <zstd-jni.version>1.5.0-4</zstd-jni.version>
     <hbase-thirdparty.version>4.0.1</hbase-thirdparty.version>
+    <nimbusds.version>9.15</nimbusds.version>

Review comment:
       Looks like they have 9.15.2 also released. I assume we would want to use the latest version.

##########
File path: hbase-common/src/test/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponseTest.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+import javax.security.sasl.SaslException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, SmallTests.class})
+public class OAuthBearerClientInitialResponseTest {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(OAuthBearerClientInitialResponseTest.class);
+
+  /*
+      Test how a client would build a response
+   */
+  @Test
+  public void testBuildClientResponseToBytes() throws Exception {
+    String expectedMesssage = "n,,\u0001auth=Bearer 123.345.567\u0001nineteen=42\u0001\u0001";

Review comment:
       Are the `\u0001` characters in here from the serialization done in `OAuthBearerStringUtils`? Would be nice to use the corresponding API to generate this String.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778101875



##########
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:
       Sure. Do I need to add them to every single test class or just this one?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
taklwu commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778307814



##########
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:
       IIRC it should be for any test class.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1008860948


   Follow-up PR https://github.com/anmolnar/hbase/pull/1


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778089800



##########
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:
       Removed completely.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1005226544


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 51s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  6s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 47s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m 33s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 22s |  master passed  |
   | -0 :warning: |  patch  |  14m 40s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  3s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  3s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   9m  8s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-common generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  javadoc  |   2m 47s |  root generated 1 new + 57 unchanged - 0 fixed = 58 total (was 57)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 447m 52s |  root in the patch failed.  |
   |  |   | 497m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 02f8d6c7992b 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e96df8bf40 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/testReport/ |
   | Max. process+thread count | 4412 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779586844



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerStringUtils.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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 OAuthBearerStringUtils {
+  /**
+   *  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,

Review comment:
       It's not JSON, it's some kind of custom Sasl serialization of extensions. How can I use GSON here?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779715206



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/SaslExtensionsCallback.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Optional callback used for SASL mechanisms if any extensions need to be set
+ * in the SASL exchange.
+ */
+@InterfaceAudience.Public
+public class SaslExtensionsCallback implements Callback {

Review comment:
       Yeah, that would be my suggestion. These "extensions" feel like some extra "ability" to pass along data from the client to the server. I'm sure we could come up with somethign to use them for later on, but that complicates the SASL implementation a bit up front.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1006811704


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 26s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 26s |  master passed  |
   | +1 :green_heart: |  compile  |  10m  5s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 22s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  18m  1s |  master passed  |
   | -0 :warning: |  patch  |  13m 33s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 20s |  the patch passed  |
   | +1 :green_heart: |  compile  |  11m 18s |  the patch passed  |
   | -0 :warning: |  javac  |  11m 18s |  root generated 1 new + 1628 unchanged - 0 fixed = 1629 total (was 1628)  |
   | +1 :green_heart: |  checkstyle  |   2m 57s |  root: The patch generated 0 new + 17 unchanged - 1 fixed = 17 total (was 18)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  25m 52s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |  22m 43s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 13s |  The patch does not generate ASF License warnings.  |
   |  |   | 118m 21s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux a1e770168d8c 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7972b2ebbb |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | Max. process+thread count | 126 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/7/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778093132



##########
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:
       Seems it makes sure that errorMessage is blank if the previous error handling block was skipped to avoid any sideeffects. I don't think it's must have, but doesn't do any harm.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778079579



##########
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:
       Copied from Kafka, so yes, this should be 3.0.0 for HBase.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1005273664


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 19s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  6s |  master passed  |
   | +1 :green_heart: |  compile  |   9m  7s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 59s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  14m 42s |  master passed  |
   | -0 :warning: |  patch  |   2m 55s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  the patch passed  |
   | +1 :green_heart: |  compile  |   9m  7s |  the patch passed  |
   | -0 :warning: |  javac  |   9m  7s |  root generated 3 new + 1628 unchanged - 0 fixed = 1631 total (was 1628)  |
   | +1 :green_heart: |  checkstyle  |   2m  1s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  1s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 50s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |  15m 57s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 23s |  The patch does not generate ASF License warnings.  |
   |  |   |  94m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux c6c6a188993e 4.15.0-156-generic #163-Ubuntu SMP Thu Aug 19 23:31:58 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 70cb9b0ba9 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | Max. process+thread count | 141 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/5/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1005066031


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 48s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   9m  6s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 58s |  master passed  |
   | -0 :warning: |  patch  |  14m 57s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 14s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 14s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 59s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 190m 50s |  root in the patch passed.  |
   |  |   | 240m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e4a26493b416 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e96df8bf40 |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/testReport/ |
   | Max. process+thread count | 7037 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778396120



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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 javax.security.sasl.SaslServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This exception indicates that SASL authentication has failed. The error message
+ * in the exception indicates the actual cause of failure.
+ * <p>
+ * SASL authentication failures typically indicate invalid credentials, but
+ * could also include other failures specific to the SASL mechanism used
+ * for authentication.
+ * </p>
+ * <p><b>Note:</b>If {@link SaslServer#evaluateResponse(byte[])} throws this exception during
+ * authentication, the message from the exception will be sent to clients in the SaslAuthenticate
+ * response. Custom {@link SaslServer} implementations may throw this exception in order to
+ * provide custom error messages to clients, but should take care not to include any
+ * security-critical information in the message that should not be leaked to unauthenticated
+ * clients.
+ * </p>
+ */
+@InterfaceAudience.Public
+public class SaslAuthenticationException extends RuntimeException {

Review comment:
       Same comment here about DoNotRetryIOException, or maybe AccessDeniedException (an HBase public exception) is more appropriate if this is more for a "bad password" kind of case.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.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.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 {

Review comment:
       We typically use the `DoNotRetryIOException` through the public API to indicate an operation (especially those which might come from a Master/RegionServer) should not be intrinsically retried. Since this is a RuntimeException, it might be treated as something that is non-retriable already.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/SaslExtensionsCallback.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Optional callback used for SASL mechanisms if any extensions need to be set
+ * in the SASL exchange.
+ */
+@InterfaceAudience.Public

Review comment:
       Private

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/SaslExtensions.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.auth;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Public

Review comment:
       Private here too

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own
+ * {@link OAuthBearerValidatorCallback} irregardless of provided extensions, as they are inherently
+ * insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * @return {@link SaslExtensions} consisting of the unvalidated extension names and values that
+   *   were sent by the client
+   */
+  public SaslExtensions inputExtensions() {
+    return inputExtensions;
+  }
+
+  /**
+   * @return an unmodifiable {@link Map} consisting of the validated and recognized by the server
+   *   extension names and values.
+   */
+  public Map<String, String> validatedExtensions() {
+    return Collections.unmodifiableMap(validatedExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the name-&gt;error messages of extensions
+   *   which failed validation
+   */
+  public Map<String, String> invalidExtensions() {

Review comment:
       ```suggestion
     public Map<String, String> getInvalidExtensions() {
   ```

##########
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 javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+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 3.0.0 and, while it feels stable, it could
+ * evolve. We will try to evolve the API in a compatible manner, but we reserve
+ * the right to make breaking changes in minor releases, if necessary. We will
+ * update the {@code InterfaceStability} annotation and this notice once the API
+ * is considered stable.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerTokenCallback implements Callback {

Review comment:
       I don't think this would ever have to be public API for HBase. I think it can be Private or LimitedPrivate. Really, users would never have to know about this class in order to write an HBase application.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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 (!(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);
+      throw e;
+    } catch (Exception e) {
+      LOG.error("SASL server problem", e);
+      throw e;
+    }
+  }
+
+  @Override
+  public String getAuthorizationID() {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return tokenForNegotiatedProperty.principalName();
+  }
+
+  @Override
+  public String getMechanismName() {
+    return OAUTHBEARER_MECHANISM;
+  }
+
+  @Override
+  public Object getNegotiatedProperty(String propName) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    if (NEGOTIATED_PROPERTY_KEY_TOKEN.equals(propName)) {
+      return tokenForNegotiatedProperty;
+    }
+    if (CREDENTIAL_LIFETIME_MS_SASL_NEGOTIATED_PROPERTY_KEY.equals(propName)) {
+      return tokenForNegotiatedProperty.lifetimeMs();
+    }
+    return extensions.map().get(propName);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return complete;
+  }
+
+  @Override
+  public byte[] unwrap(byte[] incoming, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(incoming, offset, offset + len);
+  }
+
+  @Override
+  public byte[] wrap(byte[] outgoing, int offset, int len) {
+    if (!complete) {
+      throw new IllegalStateException("Authentication exchange has not completed");
+    }
+    return Arrays.copyOfRange(outgoing, offset, offset + len);
+  }

Review comment:
       I thought SASL had some extra header on the data packet which unwrap and wrap were handling.  Maybe it's only important when we have SASL QOP set to integrity or confidentiality? With the bearer token approach we'll have to make sure that the data on the wire is encrypted.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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 {

Review comment:
       Should move this into hbase-server to keep clients from needing the nimbus-jose dependency, I think.

##########
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
+  public void testWithPotentiallyMultipleTokens() throws Exception {
+    OAuthBearerSaslClientAuthenticationProvider.OAuthBearerSaslClientCallbackHandler handler =
+      createCallbackHandler();
+    Subject.doAs(new Subject(), (PrivilegedExceptionAction<Void>) () -> {
+      final int maxTokens = 4;
+      final Set<Object> privateCredentials = Subject.getSubject(AccessController.getContext())
+        .getPrivateCredentials();
+      privateCredentials.clear();
+      for (int num = 1; num <= maxTokens; ++num) {
+        privateCredentials.add(createTokenWithLifetimeMillis(num));
+        OAuthBearerTokenCallback callback = new OAuthBearerTokenCallback();
+        handler.handle(new Callback[] {callback});
+        assertEquals(num, callback.token().lifetimeMs());
+      }

Review comment:
       Would be nice to then add a new `PrivateCredential` with a lesser `lifetimeMs`, just to make sure that the sorting is working properly (since you test the life times generated in sorted order, `1, 2, 3, 4`).

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own
+ * {@link OAuthBearerValidatorCallback} irregardless of provided extensions, as they are inherently
+ * insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {

Review comment:
       ```suggestion
     public OAuthBearerToken getToken() {
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own
+ * {@link OAuthBearerValidatorCallback} irregardless of provided extensions, as they are inherently
+ * insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * @return {@link SaslExtensions} consisting of the unvalidated extension names and values that
+   *   were sent by the client
+   */
+  public SaslExtensions inputExtensions() {

Review comment:
       ```suggestion
     public SaslExtensions getInputExtensions() {
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/SaslExtensions.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.auth;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Public
+public class SaslExtensions {
+  /**
+   * An "empty" instance indicating no SASL extensions
+   */
+  public static final SaslExtensions NO_SASL_EXTENSIONS =
+    new SaslExtensions(Collections.emptyMap());
+  private final Map<String, String> extensionsMap;
+
+  public SaslExtensions(Map<String, String> extensionsMap) {
+    this.extensionsMap = Collections.unmodifiableMap(new HashMap<>(extensionsMap));
+  }
+
+  /**
+   * Returns an <strong>immutable</strong> map of the extension names and their values
+   */
+  public Map<String, String> map() {

Review comment:
       ```suggestion
     public Map<String, String> getExtensions() {
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.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.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.

Review comment:
       Is the 'PLAINTEXT' comment relevant to HBase, or maybe copied from Kafka's implementation?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerStringUtils.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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 OAuthBearerStringUtils {
+  /**
+   *  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,

Review comment:
       We have a shaded version of GSON in hbase-thirdparty. That would be cleaner than rolling our serialization logic, I think.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own
+ * {@link OAuthBearerValidatorCallback} irregardless of provided extensions, as they are inherently
+ * insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * @return {@link SaslExtensions} consisting of the unvalidated extension names and values that
+   *   were sent by the client
+   */
+  public SaslExtensions inputExtensions() {
+    return inputExtensions;
+  }
+
+  /**
+   * @return an unmodifiable {@link Map} consisting of the validated and recognized by the server
+   *   extension names and values.
+   */
+  public Map<String, String> validatedExtensions() {
+    return Collections.unmodifiableMap(validatedExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the name-&gt;error messages of extensions
+   *   which failed validation
+   */
+  public Map<String, String> invalidExtensions() {
+    return Collections.unmodifiableMap(invalidExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the extensions that have neither been
+   *   validated nor invalidated
+   */
+  public Map<String, String> ignoredExtensions() {

Review comment:
       ```suggestion
     public Map<String, String> getIgnoredExtensions() {
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own
+ * {@link OAuthBearerValidatorCallback} irregardless of provided extensions, as they are inherently
+ * insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * @return {@link SaslExtensions} consisting of the unvalidated extension names and values that
+   *   were sent by the client
+   */
+  public SaslExtensions inputExtensions() {
+    return inputExtensions;
+  }
+
+  /**
+   * @return an unmodifiable {@link Map} consisting of the validated and recognized by the server
+   *   extension names and values.
+   */
+  public Map<String, String> validatedExtensions() {
+    return Collections.unmodifiableMap(validatedExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the name-&gt;error messages of extensions
+   *   which failed validation
+   */
+  public Map<String, String> invalidExtensions() {
+    return Collections.unmodifiableMap(invalidExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the extensions that have neither been
+   *   validated nor invalidated
+   */
+  public Map<String, String> ignoredExtensions() {
+    return Collections.unmodifiableMap(
+      subtractMap(subtractMap(inputExtensions.map(), invalidExtensions), validatedExtensions));
+  }
+
+  /**
+   * Validates a specific extension in the original {@code inputExtensions} map
+   * @param extensionName - the name of the extension which was validated
+   */
+  public void valid(String extensionName) {

Review comment:
       ```suggestion
     public void storeAsValid(String extensionName) {
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own
+ * {@link OAuthBearerValidatorCallback} irregardless of provided extensions, as they are inherently
+ * insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * @return {@link SaslExtensions} consisting of the unvalidated extension names and values that
+   *   were sent by the client
+   */
+  public SaslExtensions inputExtensions() {
+    return inputExtensions;
+  }
+
+  /**
+   * @return an unmodifiable {@link Map} consisting of the validated and recognized by the server
+   *   extension names and values.
+   */
+  public Map<String, String> validatedExtensions() {
+    return Collections.unmodifiableMap(validatedExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the name-&gt;error messages of extensions
+   *   which failed validation
+   */
+  public Map<String, String> invalidExtensions() {
+    return Collections.unmodifiableMap(invalidExtensions);
+  }
+
+  /**
+   * @return An immutable {@link Map} consisting of the extensions that have neither been
+   *   validated nor invalidated
+   */
+  public Map<String, String> ignoredExtensions() {
+    return Collections.unmodifiableMap(
+      subtractMap(subtractMap(inputExtensions.map(), invalidExtensions), validatedExtensions));
+  }
+
+  /**
+   * Validates a specific extension in the original {@code inputExtensions} map
+   * @param extensionName - the name of the extension which was validated
+   */
+  public void valid(String extensionName) {
+    if (!inputExtensions.map().containsKey(extensionName)) {
+      throw new IllegalArgumentException(
+        String.format("Extension %s was not found in the original extensions", extensionName));
+    }
+    validatedExtensions.put(extensionName, inputExtensions.map().get(extensionName));
+  }
+  /**
+   * Set the error value for a specific extension key-value pair if validation has failed
+   *
+   * @param invalidExtensionName
+   *            the mandatory extension name which caused the validation failure
+   * @param errorMessage
+   *            error message describing why the validation failed
+   */
+  public void error(String invalidExtensionName, String errorMessage) {

Review comment:
       ```suggestion
     public void storeAsError(String invalidExtensionName, String errorMessage) {
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils.subtractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A {@code Callback} for use by the {@code SaslServer} implementation when it
+ * needs to validate the SASL extensions for the OAUTHBEARER mechanism
+ * Callback handlers should use the {@link #valid(String)}
+ * method to communicate valid extensions back to the SASL server.
+ * Callback handlers should use the
+ * {@link #error(String, String)} method to communicate validation errors back to
+ * the SASL Server.
+ * As per RFC-7628 (https://tools.ietf.org/html/rfc7628#section-3.1), unknown extensions must be ignored by the server.
+ * The callback handler implementation should simply ignore unknown extensions,
+ * not calling {@link #error(String, String)} nor {@link #valid(String)}.
+ * Callback handlers should communicate other problems by raising an {@code IOException}.
+ * <p>
+ * The OAuth bearer token is provided in the callback for better context in extension validation.
+ * It is very important that token validation is done in its own
+ * {@link OAuthBearerValidatorCallback} irregardless of provided extensions, as they are inherently
+ * insecure.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerExtensionsValidatorCallback implements Callback {
+  private final OAuthBearerToken token;
+  private final SaslExtensions inputExtensions;
+  private final Map<String, String> validatedExtensions = new HashMap<>();
+  private final Map<String, String> invalidExtensions = new HashMap<>();
+
+  public OAuthBearerExtensionsValidatorCallback(OAuthBearerToken token, SaslExtensions extensions) {
+    this.token = Objects.requireNonNull(token);
+    this.inputExtensions = Objects.requireNonNull(extensions);
+  }
+
+  /**
+   * @return {@link OAuthBearerToken} the OAuth bearer token of the client
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * @return {@link SaslExtensions} consisting of the unvalidated extension names and values that
+   *   were sent by the client
+   */
+  public SaslExtensions inputExtensions() {
+    return inputExtensions;
+  }
+
+  /**
+   * @return an unmodifiable {@link Map} consisting of the validated and recognized by the server
+   *   extension names and values.
+   */
+  public Map<String, String> validatedExtensions() {

Review comment:
       ```suggestion
     public Map<String, String> getValidatedExtensions() {
   ```

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/AuthenticateCallbackHandler.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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

Review comment:
       Users would not be implementing this directly, would they? I think good to start this off as "Private"

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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 java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.security.sasl.SaslException;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * OAuthBearer SASL client's initial message to the server.
+ *
+ * This class has been copy-and-pasted from Kafka codebase.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerClientInitialResponse {
+  static final String SEPARATOR = "\u0001";
+
+  private static final String SASLNAME = "(?:[\\x01-\\x7F&&[^=,]]|=2C|=3D)+";
+  private static final String KEY = "[A-Za-z]+";
+  private static final String VALUE = "[\\x21-\\x7E \t\r\n]+";
+
+  private static final String KVPAIRS = String.format("(%s=%s%s)*", KEY, VALUE, SEPARATOR);
+  private static final Pattern AUTH_PATTERN =
+    Pattern.compile("(?<scheme>[\\w]+)[ ]+(?<token>[-_\\.a-zA-Z0-9]+)");
+  private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile(
+    String.format("n,(a=(?<authzid>%s))?,%s(?<kvpairs>%s)%s",
+      SASLNAME, SEPARATOR, KVPAIRS, SEPARATOR));
+  public static final String AUTH_KEY = "auth";
+
+  private final String tokenValue;
+  private final String authorizationId;
+  private final SaslExtensions saslExtensions;
+
+  public static final Pattern EXTENSION_KEY_PATTERN = Pattern.compile(KEY);
+  public static final Pattern EXTENSION_VALUE_PATTERN = Pattern.compile(VALUE);
+
+  public OAuthBearerClientInitialResponse(byte[] response) throws SaslException {
+    String responseMsg = new String(response, StandardCharsets.UTF_8);
+    Matcher matcher = CLIENT_INITIAL_RESPONSE_PATTERN.matcher(responseMsg);

Review comment:
       Some `LOG.trace()` calls in here would go a long way to debug why we couldn't parse the response.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/SaslExtensionsCallback.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Optional callback used for SASL mechanisms if any extensions need to be set
+ * in the SASL exchange.
+ */
+@InterfaceAudience.Public
+public class SaslExtensionsCallback implements Callback {

Review comment:
       Not clear to me what these "Extensions" are used for. Do we need them for the basic functionality?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerClientInitialResponse.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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 java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.security.sasl.SaslException;
+import org.apache.hadoop.hbase.security.auth.SaslExtensions;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerStringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * OAuthBearer SASL client's initial message to the server.
+ *
+ * This class has been copy-and-pasted from Kafka codebase.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerClientInitialResponse {
+  static final String SEPARATOR = "\u0001";
+
+  private static final String SASLNAME = "(?:[\\x01-\\x7F&&[^=,]]|=2C|=3D)+";
+  private static final String KEY = "[A-Za-z]+";
+  private static final String VALUE = "[\\x21-\\x7E \t\r\n]+";
+
+  private static final String KVPAIRS = String.format("(%s=%s%s)*", KEY, VALUE, SEPARATOR);
+  private static final Pattern AUTH_PATTERN =
+    Pattern.compile("(?<scheme>[\\w]+)[ ]+(?<token>[-_\\.a-zA-Z0-9]+)");
+  private static final Pattern CLIENT_INITIAL_RESPONSE_PATTERN = Pattern.compile(
+    String.format("n,(a=(?<authzid>%s))?,%s(?<kvpairs>%s)%s",
+      SASLNAME, SEPARATOR, KVPAIRS, SEPARATOR));
+  public static final String AUTH_KEY = "auth";
+
+  private final String tokenValue;
+  private final String authorizationId;
+  private final SaslExtensions saslExtensions;
+
+  public static final Pattern EXTENSION_KEY_PATTERN = Pattern.compile(KEY);
+  public static final Pattern EXTENSION_VALUE_PATTERN = Pattern.compile(VALUE);
+
+  public OAuthBearerClientInitialResponse(byte[] response) throws SaslException {
+    String responseMsg = new String(response, StandardCharsets.UTF_8);
+    Matcher matcher = CLIENT_INITIAL_RESPONSE_PATTERN.matcher(responseMsg);
+    if (!matcher.matches()) {
+      throw new SaslException("Invalid OAUTHBEARER client first message");
+    }
+    String authzid = matcher.group("authzid");
+    this.authorizationId = authzid == null ? "" : authzid;
+    String kvPairs = matcher.group("kvpairs");
+    Map<String, String> properties = OAuthBearerStringUtils.parseMap(kvPairs, "=", SEPARATOR);
+    String auth = properties.get(AUTH_KEY);
+    if (auth == null) {
+      throw new SaslException("Invalid OAUTHBEARER client first message: 'auth' not specified");
+    }
+    properties.remove(AUTH_KEY);
+    SaslExtensions extensions = new SaslExtensions(properties);
+    validateExtensions(extensions);
+    this.saslExtensions = extensions;
+
+    Matcher authMatcher = AUTH_PATTERN.matcher(auth);
+    if (!authMatcher.matches()) {
+      throw new SaslException("Invalid OAUTHBEARER client first message: invalid 'auth' format");
+    }
+    if (!"bearer".equalsIgnoreCase(authMatcher.group("scheme"))) {
+      String msg = String.format("Invalid scheme in OAUTHBEARER client first message: %s",
+        matcher.group("scheme"));
+      throw new SaslException(msg);
+    }
+    this.tokenValue = authMatcher.group("token");
+  }
+
+  /**
+   * Constructor
+   *
+   * @param tokenValue
+   *            the mandatory token value
+   * @param extensions
+   *            the optional extensions
+   * @throws SaslException
+   *             if any extension name or value fails to conform to the required
+   *             regular expression as defined by the specification, or if the
+   *             reserved {@code auth} appears as a key
+   */
+  public OAuthBearerClientInitialResponse(String tokenValue, SaslExtensions extensions)
+    throws SaslException {
+    this(tokenValue, "", extensions);
+  }
+
+  /**
+   * Constructor
+   *
+   * @param tokenValue
+   *            the mandatory token value
+   * @param authorizationId
+   *            the optional authorization ID
+   * @param extensions
+   *            the optional extensions
+   * @throws SaslException
+   *             if any extension name or value fails to conform to the required
+   *             regular expression as defined by the specification, or if the
+   *             reserved {@code auth} appears as a key
+   */
+  public OAuthBearerClientInitialResponse(String tokenValue, String authorizationId,
+    SaslExtensions extensions) throws SaslException {
+    this.tokenValue = Objects.requireNonNull(tokenValue, "token value must not be null");
+    this.authorizationId = authorizationId == null ? "" : authorizationId;
+    validateExtensions(extensions);
+    this.saslExtensions = extensions != null ? extensions : SaslExtensions.NO_SASL_EXTENSIONS;
+  }
+
+  /**
+   * Return the always non-null extensions
+   *
+   * @return the always non-null extensions
+   */
+  public SaslExtensions extensions() {
+    return saslExtensions;
+  }
+
+  public byte[] toBytes() {
+    String authzid = authorizationId.isEmpty() ? "" : "a=" + authorizationId;
+    String extensions = extensionsMessage();
+    if (extensions.length() > 0) {
+      extensions = SEPARATOR + extensions;
+    }
+
+    String message = String.format("n,%s,%sauth=Bearer %s%s%s%s", authzid,
+      SEPARATOR, tokenValue, extensions, SEPARATOR, SEPARATOR);
+
+    return message.getBytes(StandardCharsets.UTF_8);

Review comment:
       nit: Could use the helper `Bytes.toBytes(message)` we have in HBase.

##########
File path: hbase-common/pom.xml
##########
@@ -255,6 +255,10 @@
       <artifactId>kerb-simplekdc</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.nimbusds</groupId>
+      <artifactId>nimbus-jose-jwt</artifactId>

Review comment:
       It would be good if we could keep this on the server classpath only. Do we need this in hbase-common?

##########
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 javax.security.auth.callback.Callback;
+import org.apache.commons.lang3.StringUtils;
+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 3.0.0 and, while it feels stable, it could
+ * evolve. We will try to evolve the API in a compatible manner, but we reserve
+ * the right to make breaking changes in minor releases, if necessary. We will
+ * update the {@code InterfaceStability} annotation and this notice once the API
+ * is considered stable.
+ */
+@InterfaceAudience.Public
+public class OAuthBearerTokenCallback implements Callback {
+  private OAuthBearerToken token = null;
+  private String errorCode = null;
+  private String errorDescription = null;
+  private String errorUri = null;
+
+  /**
+   * Return the (potentially null) token
+   *
+   * @return the (potentially null) token
+   */
+  public OAuthBearerToken token() {
+    return token;
+  }
+
+  /**
+   * Return the optional (but always non-empty if not null) error code as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the optional (but always non-empty if not null) error code
+   */
+  public String errorCode() {
+    return errorCode;
+  }
+
+  /**
+   * Return the (potentially null) error description as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error description
+   */
+  public String errorDescription() {
+    return errorDescription;
+  }
+
+  /**
+   * Return the (potentially null) error URI as per
+   * <a href="https://tools.ietf.org/html/rfc6749#section-5.2">RFC 6749: The OAuth
+   * 2.0 Authorization Framework</a>.
+   *
+   * @return the (potentially null) error URI
+   */
+  public String errorUri() {
+    return errorUri;
+  }
+
+  /**
+   * Set the token. All error-related values are cleared.
+   *
+   * @param token
+   *            the optional token to set
+   */
+  public void token(OAuthBearerToken token) {

Review comment:
       Should these be mutable? For one Callback instance, we would set the token the first time and potentially call `error()` later?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r781271576



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/SaslExtensionsCallback.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Optional callback used for SASL mechanisms if any extensions need to be set
+ * in the SASL exchange.
+ */
+@InterfaceAudience.Public
+public class SaslExtensionsCallback implements Callback {

Review comment:
       Sure, I'll remove it.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778084206



##########
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:
       In this case checking the `jwkSet` would be enough. I was thinking of this initialization thing a lot and doesn't have a strong opinion. Kafka follows this "configured" pattern pretty much everywhere and I'm not sure if there's convention in HBase for this.
   
   Addig a constructor and expecting that `jwkSet` is built by the instantiator would also be an option.
   
   What would you recommend?
   




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1006201954


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 12s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 36s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 41s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |  10m 27s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 57s |  master passed  |
   | -0 :warning: |  patch  |  16m 17s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 55s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |  10m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-common generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  javadoc  |   3m  2s |  root generated 1 new + 57 unchanged - 0 fixed = 58 total (was 57)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 446m  4s |  root in the patch passed.  |
   |  |   | 499m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux f6b45885180d 4.15.0-153-generic #160-Ubuntu SMP Thu Jul 29 06:54:29 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / baeb51ff8a |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/testReport/ |
   | Max. process+thread count | 4510 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779579220



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/SaslAuthenticationException.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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 javax.security.sasl.SaslServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This exception indicates that SASL authentication has failed. The error message
+ * in the exception indicates the actual cause of failure.
+ * <p>
+ * SASL authentication failures typically indicate invalid credentials, but
+ * could also include other failures specific to the SASL mechanism used
+ * for authentication.
+ * </p>
+ * <p><b>Note:</b>If {@link SaslServer#evaluateResponse(byte[])} throws this exception during
+ * authentication, the message from the exception will be sent to clients in the SaslAuthenticate
+ * response. Custom {@link SaslServer} implementations may throw this exception in order to
+ * provide custom error messages to clients, but should take care not to include any
+ * security-critical information in the message that should not be leaked to unauthenticated
+ * clients.
+ * </p>
+ */
+@InterfaceAudience.Public
+public class SaslAuthenticationException extends RuntimeException {

Review comment:
       Same restriction applies here: the method where this is thrown cannot throw anything than `SaslException`.




-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1006064412


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 36s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 32s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 19s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 56s |  master passed  |
   | -0 :warning: |  patch  |  14m 18s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 15s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 15s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 13s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 56s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 191m 38s |  root in the patch passed.  |
   |  |   | 239m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3c7f2bd4a469 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / baeb51ff8a |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/testReport/ |
   | Max. process+thread count | 7035 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



[GitHub] [hbase] anmolnar edited a comment on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
anmolnar edited a comment on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1008860948


   Follow-up PR https://github.com/apache/hbase/pull/4019


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1008857858


   Closing this PR due to feature branch request.


-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1004935063


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m  1s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  master passed  |
   | +1 :green_heart: |  compile  |   9m  1s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m  0s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  14m 41s |  master passed  |
   | -0 :warning: |  patch  |   2m 52s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 52s |  the patch passed  |
   | +1 :green_heart: |  compile  |   9m  0s |  the patch passed  |
   | -0 :warning: |  javac  |   9m  0s |  root generated 11 new + 1628 unchanged - 0 fixed = 1639 total (was 1628)  |
   | -0 :warning: |  checkstyle  |   2m  0s |  root: The patch generated 2 new + 17 unchanged - 0 fixed = 19 total (was 17)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 48s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |  15m 57s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  96m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 190f6f2c74a8 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e96df8bf40 |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | Max. process+thread count | 141 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/4/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779673170



##########
File path: hbase-common/pom.xml
##########
@@ -255,6 +255,10 @@
       <artifactId>kerb-simplekdc</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.nimbusds</groupId>
+      <artifactId>nimbus-jose-jwt</artifactId>

Review comment:
       It's done.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779716278



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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 {

Review comment:
       Thanks! I'll re-review with this in mind.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778218640



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/security/provider/OAuthBearerSaslServerAuthenticationProvider.java
##########
@@ -0,0 +1,99 @@
+/*
+ *  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 java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.security.auth.AuthenticateCallbackHandler;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.OAuthBearerSaslServerProvider;
+import org.apache.hadoop.hbase.security.oauthbearer.internals.knox.OAuthBearerSignedJwtValidatorCallbackHandler;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class OAuthBearerSaslServerAuthenticationProvider
+    extends OAuthBearerSaslAuthenticationProvider
+    implements SaslServerAuthenticationProvider {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    OAuthBearerSaslServerAuthenticationProvider.class);
+  private Configuration hbaseConfiguration;
+  private boolean initialized = false;
+
+  static {
+    OAuthBearerSaslServerProvider.initialize(); // not part of public API
+    LOG.info("OAuthBearer SASL server provider has been initialized");
+  }
+
+  @Override public void init(Configuration conf) throws IOException {
+    this.hbaseConfiguration = conf;
+    this.initialized = true;
+  }
+
+  @Override public AttemptingUserProvidingSaslServer createServer(
+    SecretManager<TokenIdentifier> secretManager, Map<String, String> saslProps)
+    throws IOException {
+
+    if (!initialized) {
+      throw new RuntimeException(
+        "OAuthBearerSaslServerAuthenticationProvider must be initialized first.");
+    }
+
+    UserGroupInformation current = UserGroupInformation.getCurrentUser();
+    String fullName = current.getUserName();
+    LOG.debug("Server's OAuthBearer user name is {}", fullName);
+    LOG.debug("OAuthBearer saslProps = {}", saslProps);
+
+    try {
+      return current.doAs(new PrivilegedExceptionAction<AttemptingUserProvidingSaslServer>() {
+        @Override
+        public AttemptingUserProvidingSaslServer run() throws SaslException {
+          AuthenticateCallbackHandler callbackHandler =
+            new OAuthBearerSignedJwtValidatorCallbackHandler();
+          callbackHandler.configure(hbaseConfiguration, getSaslAuthMethod().getSaslMechanism(),
+            saslProps);
+          return new AttemptingUserProvidingSaslServer(Sasl.createSaslServer(
+            getSaslAuthMethod().getSaslMechanism(), null, null, saslProps,
+            callbackHandler), () -> null);
+        }
+      });
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new RuntimeException("Failed to construct OAUTHBEARER SASL server");

Review comment:
       Do you have a specific recommendation? Why not RuntimeException?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1007784888


   Alright! I was actually able to test this out today using Knox. I think there are a couple of high level things we need to figure out
   
   * HBase clients will expect that renewals transparently happen. Either, we need a renewer thread in hbase to get a new bearer token before it expires (I think this is possible, but we'd have to know where to get the new one from). Otherwise, we'd have to think about usign the bearer token to get an hbase delegation token (which seems like too many tokens, tbh)
   * How will users provide the bearer token into their HBase client? Environment variable? Well-known file?
   * I tried enabling the RPC encryption from HBASE-16414 but regrettably can still see plaintext data going over the wire. Maybe that's just a bug in this patch, or maybe it's a bigger HBase SASL issue. Either way, we need encryption if we enable this auth'n feature.
   * I would like to see a standalone (no external service dependency) test included in hbase-examples, rather than just a client. However, I don't know of a JWT-providing server we could easily embedded into a test. Maybe knox could do this, or maybe nimbus has some testing server?
   * Need to get some additions to the hbase book.
   
   Now, given how big this patch is already, I think I'd suggest we work through these on a feature branch rather than try to do them in a single commit. WDYT, Andor? I think this approach would let us do some iteration more easily.


-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-991053512


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 40s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  9s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 56s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  9s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  9s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 22s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-common generated 1 new + 12 unchanged - 0 fixed = 13 total (was 12)  |
   | -0 :warning: |  javadoc  |   2m 49s |  root generated 1 new + 171 unchanged - 0 fixed = 172 total (was 171)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 186m 15s |  root in the patch passed.  |
   |  |   | 233m 34s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d3f2cba9fdbd 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / b5cf3cdc5c |
   | Default Java | AdoptOpenJDK-11.0.10+9 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/testReport/ |
   | Max. process+thread count | 7242 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-991129209


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  7s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 49s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 23s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m  6s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 47s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   8m 18s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-common generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -0 :warning: |  javadoc  |   2m 14s |  root generated 2 new + 54 unchanged - 0 fixed = 56 total (was 54)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  | 285m 45s |  root in the patch failed.  |
   |  |   | 328m 37s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 48f73397b479 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / b5cf3cdc5c |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/testReport/ |
   | Max. process+thread count | 4488 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/1/console |
   | versions | git=2.17.1 maven=3.6.3 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
taklwu commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778306068



##########
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:
       I lost the original commit and it seems fine now after the second look at the `configure(Configuration configs, JWKSet jwkSet)` method again. basically this pattern is about configuring multiple parameters with in the single constructor-like method, so I would say it's fine now and we don't need to change it. 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778093469



##########
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:
       Good catch!




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779577162



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/IllegalSaslStateException.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.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.

Review comment:
       I don't think so, removed it.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779630173



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/auth/SaslExtensionsCallback.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.Objects;
+import javax.security.auth.callback.Callback;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Optional callback used for SASL mechanisms if any extensions need to be set
+ * in the SASL exchange.
+ */
+@InterfaceAudience.Public
+public class SaslExtensionsCallback implements Callback {

Review comment:
       No, I havne't used this feature during my testing at all. Do you think we should remove it completely?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
joshelser commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1007784888


   Alright! I was actually able to test this out today using Knox. I think there are a couple of high level things we need to figure out
   
   * HBase clients will expect that renewals transparently happen. Either, we need a renewer thread in hbase to get a new bearer token before it expires (I think this is possible, but we'd have to know where to get the new one from). Otherwise, we'd have to think about usign the bearer token to get an hbase delegation token (which seems like too many tokens, tbh)
   * How will users provide the bearer token into their HBase client? Environment variable? Well-known file?
   * I tried enabling the RPC encryption from HBASE-16414 but regrettably can still see plaintext data going over the wire. Maybe that's just a bug in this patch, or maybe it's a bigger HBase SASL issue. Either way, we need encryption if we enable this auth'n feature.
   * I would like to see a standalone (no external service dependency) test included in hbase-examples, rather than just a client. However, I don't know of a JWT-providing server we could easily embedded into a test. Maybe knox could do this, or maybe nimbus has some testing server?
   * Need to get some additions to the hbase book.
   
   Now, given how big this patch is already, I think I'd suggest we work through these on a feature branch rather than try to do them in a single commit. WDYT, Andor? I think this approach would let us do some iteration more easily.


-- 
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



[GitHub] [hbase] Apache-HBase commented on pull request #3934: HBASE-26553 OAuth Bearer authentication mech plugin for SASL

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#issuecomment-1005967634


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  1s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  master passed  |
   | +1 :green_heart: |  compile  |   9m  1s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m  0s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  14m 52s |  master passed  |
   | -0 :warning: |  patch  |   2m 52s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  the patch passed  |
   | +1 :green_heart: |  compile  |   8m 57s |  the patch passed  |
   | -0 :warning: |  javac  |   8m 57s |  root generated 3 new + 1628 unchanged - 0 fixed = 1631 total (was 1628)  |
   | +1 :green_heart: |  checkstyle  |   1m 57s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  2s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 28s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.2 3.3.1.  |
   | +1 :green_heart: |  spotbugs  |  15m 47s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   1m 23s |  The patch does not generate ASF License warnings.  |
   |  |   |  92m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/3934 |
   | Optional Tests | dupname asflicense javac spotbugs hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux a8ee68066013 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / baeb51ff8a |
   | Default Java | AdoptOpenJDK-1.8.0_282-b08 |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/artifact/yetus-general-check/output/diff-compile-javac-root.txt |
   | Max. process+thread count | 141 (vs. ulimit of 30000) |
   | modules | C: hbase-common hbase-client hbase-resource-bundle hbase-server hbase-examples . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-3934/6/console |
   | versions | git=2.17.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.12.0 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r779672461



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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 {

Review comment:
       I just submitted a commit which has refactoring on this front. I tried to move every logic to server which not strictly needed in common. I think I could do this with the client side as well leaving the bare minimum stuff in hbase-common.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/security/oauthbearer/internals/OAuthBearerSaslServer.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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 com.nimbusds.jose.shaded.json.JSONObject;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+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.commons.lang3.StringUtils;
+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.OAuthBearerStringUtils;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerToken;
+import org.apache.hadoop.hbase.security.oauthbearer.OAuthBearerValidatorCallback;
+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 {

Review comment:
       I just submitted a commit which has refactoring on this front. I tried to move every logic to server which not strictly needed in common. I think I could do this with the client side as well leaving the bare minimum in hbase-common.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on a change in pull request #3934:
URL: https://github.com/apache/hbase/pull/3934#discussion_r778085067



##########
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:
       Renamed to `OAuthBearerStringUtils`




-- 
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