You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by dw...@apache.org on 2022/09/19 22:56:17 UTC

[iceberg] branch master updated: REST: implement handling of OAuth error responses (#5698)

This is an automated email from the ASF dual-hosted git repository.

dweeks pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new c293af2f1d REST: implement handling of OAuth error responses (#5698)
c293af2f1d is described below

commit c293af2f1d962f44bb5b67a20ef3c67bd5823a38
Author: Bryan Keller <br...@gmail.com>
AuthorDate: Mon Sep 19 15:56:08 2022 -0700

    REST: implement handling of OAuth error responses (#5698)
    
    * WIP error handling for OAuth
    
    * cleanup
    
    * tests
    
    * handle non-oauth errors in oauth
    
    * add comment
    
    * allow null fields
    
    * more tests
    
    * more cleanup
    
    * remove unneeded precondition checks
    
    * Fix test
    
    * use assert4j
---
 .../java/org/apache/iceberg/rest/ErrorHandler.java |  26 +++++
 .../org/apache/iceberg/rest/ErrorHandlers.java     | 122 ++++++++++++++++-----
 .../java/org/apache/iceberg/rest/HTTPClient.java   |  27 ++---
 .../java/org/apache/iceberg/rest/RESTClient.java   |  33 ++----
 .../org/apache/iceberg/rest/RESTErrorResponse.java |  22 ++++
 .../org/apache/iceberg/rest/RESTSerializers.java   |  22 ++++
 .../apache/iceberg/rest/RESTTableOperations.java   |   4 +-
 .../apache/iceberg/rest/auth/OAuth2Properties.java |   8 ++
 .../org/apache/iceberg/rest/auth/OAuth2Util.java   |   6 +-
 .../iceberg/rest/responses/ErrorResponse.java      |   4 +-
 .../rest/responses/ErrorResponseParser.java        |  16 +--
 .../iceberg/rest/responses/OAuthErrorResponse.java | 100 +++++++++++++++++
 .../rest/responses/OAuthErrorResponseParser.java   |  83 ++++++++++++++
 .../apache/iceberg/rest/RESTCatalogAdapter.java    |  20 ++--
 .../org/apache/iceberg/rest/TestHTTPClient.java    |  46 ++++----
 .../org/apache/iceberg/rest/TestRESTCatalog.java   |   6 +-
 .../rest/responses/TestErrorResponseParser.java    |   9 +-
 .../responses/TestOAuthErrorResponseParser.java    | 114 +++++++++++++++++++
 open-api/rest-catalog-open-api.yaml                |   6 +-
 19 files changed, 538 insertions(+), 136 deletions(-)

diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandler.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandler.java
new file mode 100644
index 0000000000..59600d1a53
--- /dev/null
+++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandler.java
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.rest;
+
+public interface ErrorHandler {
+
+  RESTErrorResponse parseResponse(int code, String json);
+
+  void handle(RESTErrorResponse restError);
+}
diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java
index b800106ce1..1a31065886 100644
--- a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java
+++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iceberg.rest;
 
-import java.util.function.Consumer;
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.exceptions.BadRequestException;
 import org.apache.iceberg.exceptions.CommitFailedException;
@@ -30,7 +29,11 @@ import org.apache.iceberg.exceptions.NotAuthorizedException;
 import org.apache.iceberg.exceptions.RESTException;
 import org.apache.iceberg.exceptions.ServiceFailureException;
 import org.apache.iceberg.exceptions.ServiceUnavailableException;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
 import org.apache.iceberg.rest.responses.ErrorResponse;
+import org.apache.iceberg.rest.responses.ErrorResponseParser;
+import org.apache.iceberg.rest.responses.OAuthErrorResponse;
+import org.apache.iceberg.rest.responses.OAuthErrorResponseParser;
 
 /**
  * A set of consumers to handle errors for requests for table entities or for namespace entities, to
@@ -40,20 +43,33 @@ public class ErrorHandlers {
 
   private ErrorHandlers() {}
 
-  public static Consumer<ErrorResponse> namespaceErrorHandler() {
-    return baseNamespaceErrorHandler().andThen(defaultErrorHandler());
+  public static ErrorHandler namespaceErrorHandler() {
+    return NamespaceErrorHandler.INSTANCE;
   }
 
-  public static Consumer<ErrorResponse> tableErrorHandler() {
-    return baseTableErrorHandler().andThen(defaultErrorHandler());
+  public static ErrorHandler tableErrorHandler() {
+    return TableErrorHandler.INSTANCE;
   }
 
-  public static Consumer<ErrorResponse> tableCommitHandler() {
-    return baseCommitErrorHandler().andThen(defaultErrorHandler());
+  public static ErrorHandler tableCommitHandler() {
+    return CommitErrorHandler.INSTANCE;
   }
 
-  private static Consumer<ErrorResponse> baseCommitErrorHandler() {
-    return error -> {
+  public static ErrorHandler defaultErrorHandler() {
+    return DefaultErrorHandler.INSTANCE;
+  }
+
+  public static ErrorHandler oauthErrorHandler() {
+    return OAuthErrorHandler.INSTANCE;
+  }
+
+  /** Table commit error handler. */
+  private static class CommitErrorHandler extends DefaultErrorHandler {
+    private static final ErrorHandler INSTANCE = new CommitErrorHandler();
+
+    @Override
+    public void handle(RESTErrorResponse restError) {
+      ErrorResponse error = (ErrorResponse) restError;
       switch (error.code()) {
         case 404:
           throw new NoSuchTableException("%s", error.message());
@@ -64,15 +80,18 @@ public class ErrorHandlers {
           throw new CommitStateUnknownException(
               new ServiceFailureException("Service failed: %s: %s", error.code(), error.message()));
       }
-    };
+
+      super.handle(restError);
+    }
   }
 
-  /**
-   * Table level error handlers. Should be chained wih the {@link #defaultErrorHandler}, which takes
-   * care of common cases.
-   */
-  private static Consumer<ErrorResponse> baseTableErrorHandler() {
-    return error -> {
+  /** Table level error handler. */
+  private static class TableErrorHandler extends DefaultErrorHandler {
+    private static final ErrorHandler INSTANCE = new TableErrorHandler();
+
+    @Override
+    public void handle(RESTErrorResponse restError) {
+      ErrorResponse error = (ErrorResponse) restError;
       switch (error.code()) {
         case 404:
           if (NoSuchNamespaceException.class.getSimpleName().equals(error.type())) {
@@ -83,15 +102,18 @@ public class ErrorHandlers {
         case 409:
           throw new AlreadyExistsException("%s", error.message());
       }
-    };
+
+      super.handle(restError);
+    }
   }
 
-  /**
-   * Request error handlers specifically for CRUD ops on namespaces. Should be chained wih the
-   * {@link #defaultErrorHandler}, which takes care of common cases.
-   */
-  private static Consumer<ErrorResponse> baseNamespaceErrorHandler() {
-    return error -> {
+  /** Request error handler specifically for CRUD ops on namespaces. */
+  private static class NamespaceErrorHandler extends DefaultErrorHandler {
+    private static final ErrorHandler INSTANCE = new NamespaceErrorHandler();
+
+    @Override
+    public void handle(RESTErrorResponse restError) {
+      ErrorResponse error = (ErrorResponse) restError;
       switch (error.code()) {
         case 404:
           throw new NoSuchNamespaceException("%s", error.message());
@@ -100,15 +122,26 @@ public class ErrorHandlers {
         case 422:
           throw new RESTException("Unable to process: %s", error.message());
       }
-    };
+
+      super.handle(restError);
+    }
   }
 
   /**
    * Request error handler that handles the common cases that are included with all responses, such
    * as 400, 500, etc.
    */
-  public static Consumer<ErrorResponse> defaultErrorHandler() {
-    return error -> {
+  private static class DefaultErrorHandler implements ErrorHandler {
+    private static final ErrorHandler INSTANCE = new DefaultErrorHandler();
+
+    @Override
+    public RESTErrorResponse parseResponse(int code, String json) {
+      return ErrorResponseParser.fromJson(json);
+    }
+
+    @Override
+    public void handle(RESTErrorResponse restError) {
+      ErrorResponse error = (ErrorResponse) restError;
       switch (error.code()) {
         case 400:
           throw new BadRequestException("Malformed request: %s", error.message());
@@ -128,6 +161,41 @@ public class ErrorHandlers {
       }
 
       throw new RESTException("Unable to process: %s", error.message());
-    };
+    }
+  }
+
+  private static class OAuthErrorHandler implements ErrorHandler {
+    private static final ErrorHandler INSTANCE = new OAuthErrorHandler();
+    private static final String SERVER_ERROR = "server_error";
+
+    @Override
+    public RESTErrorResponse parseResponse(int code, String json) {
+      if (code == 400 || code == 401) {
+        return OAuthErrorResponseParser.fromJson(json);
+      }
+      return OAuthErrorResponse.builder()
+          .withError(SERVER_ERROR)
+          .withErrorDescription(json)
+          .build();
+    }
+
+    @Override
+    public void handle(RESTErrorResponse restError) {
+      if (restError instanceof OAuthErrorResponse) {
+        OAuthErrorResponse error = (OAuthErrorResponse) restError;
+        switch (error.error()) {
+          case OAuth2Properties.INVALID_CLIENT_ERROR:
+            throw new NotAuthorizedException("Not authorized: %s", error.errorDescription());
+          case OAuth2Properties.INVALID_REQUEST_ERROR:
+          case OAuth2Properties.INVALID_GRANT_ERROR:
+          case OAuth2Properties.UNAUTHORIZED_CLIENT_ERROR:
+          case OAuth2Properties.UNSUPPORTED_GRANT_TYPE_ERROR:
+          case OAuth2Properties.INVALID_SCOPE_ERROR:
+            throw new BadRequestException("Malformed request: %s", error.errorDescription());
+          default:
+            throw new RESTException("Unable to process: %s", error.errorDescription());
+        }
+      }
+    }
   }
 }
diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java
index 154ef2a6c2..531b27fd0f 100644
--- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java
+++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java
@@ -25,7 +25,6 @@ import java.io.UncheckedIOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Map;
-import java.util.function.Consumer;
 import org.apache.hc.client5.http.classic.methods.HttpUriRequest;
 import org.apache.hc.client5.http.classic.methods.HttpUriRequestBase;
 import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
@@ -46,7 +45,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.rest.responses.ErrorResponse;
-import org.apache.iceberg.rest.responses.ErrorResponseParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -88,7 +86,7 @@ public class HTTPClient implements RESTClient {
         || code == HttpStatus.SC_NO_CONTENT;
   }
 
-  private static ErrorResponse buildDefaultErrorResponse(CloseableHttpResponse response) {
+  private static RESTErrorResponse buildDefaultErrorResponse(CloseableHttpResponse response) {
     String responseReason = response.getReasonPhrase();
     String message =
         responseReason != null && !responseReason.isEmpty()
@@ -105,12 +103,12 @@ public class HTTPClient implements RESTClient {
   // Process a failed response through the provided errorHandler, and throw a RESTException if the
   // provided error handler doesn't already throw.
   private static void throwFailure(
-      CloseableHttpResponse response, String responseBody, Consumer<ErrorResponse> errorHandler) {
-    ErrorResponse errorResponse = null;
+      CloseableHttpResponse response, String responseBody, ErrorHandler errorHandler) {
+    RESTErrorResponse errorResponse = null;
 
     if (responseBody != null) {
       try {
-        errorResponse = ErrorResponseParser.fromJson(responseBody);
+        errorResponse = errorHandler.parseResponse(response.getCode(), responseBody);
       } catch (UncheckedIOException | IllegalArgumentException e) {
         // It's possible to receive a non-successful response that isn't a properly defined
         // ErrorResponse
@@ -129,7 +127,7 @@ public class HTTPClient implements RESTClient {
       errorResponse = buildDefaultErrorResponse(response);
     }
 
-    errorHandler.accept(errorResponse);
+    errorHandler.handle(errorResponse);
 
     // Throw an exception in case the provided error handler does not throw.
     throw new RESTException("Unhandled error: %s", errorResponse);
@@ -171,7 +169,7 @@ public class HTTPClient implements RESTClient {
       Object requestBody,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     if (path.startsWith("/")) {
       throw new RESTException(
           "Received a malformed path for a REST request: %s. Paths should not start with /", path);
@@ -227,7 +225,7 @@ public class HTTPClient implements RESTClient {
   }
 
   @Override
-  public void head(String path, Map<String, String> headers, Consumer<ErrorResponse> errorHandler) {
+  public void head(String path, Map<String, String> headers, ErrorHandler errorHandler) {
     execute(Method.HEAD, path, null, null, null, headers, errorHandler);
   }
 
@@ -237,7 +235,7 @@ public class HTTPClient implements RESTClient {
       Map<String, String> queryParams,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return execute(Method.GET, path, queryParams, null, responseType, headers, errorHandler);
   }
 
@@ -247,16 +245,13 @@ public class HTTPClient implements RESTClient {
       RESTRequest body,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return execute(Method.POST, path, null, body, responseType, headers, errorHandler);
   }
 
   @Override
   public <T extends RESTResponse> T delete(
-      String path,
-      Class<T> responseType,
-      Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      String path, Class<T> responseType, Map<String, String> headers, ErrorHandler errorHandler) {
     return execute(Method.DELETE, path, null, null, responseType, headers, errorHandler);
   }
 
@@ -266,7 +261,7 @@ public class HTTPClient implements RESTClient {
       Map<String, String> formData,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return execute(Method.POST, path, null, formData, responseType, headers, errorHandler);
   }
 
diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTClient.java b/core/src/main/java/org/apache/iceberg/rest/RESTClient.java
index 2057f23fc3..ebe54cc747 100644
--- a/core/src/main/java/org/apache/iceberg/rest/RESTClient.java
+++ b/core/src/main/java/org/apache/iceberg/rest/RESTClient.java
@@ -20,48 +20,39 @@ package org.apache.iceberg.rest;
 
 import java.io.Closeable;
 import java.util.Map;
-import java.util.function.Consumer;
 import java.util.function.Supplier;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
-import org.apache.iceberg.rest.responses.ErrorResponse;
 
 /** Interface for a basic HTTP Client for interfacing with the REST catalog. */
 public interface RESTClient extends Closeable {
 
-  default void head(
-      String path, Supplier<Map<String, String>> headers, Consumer<ErrorResponse> errorHandler) {
+  default void head(String path, Supplier<Map<String, String>> headers, ErrorHandler errorHandler) {
     head(path, headers.get(), errorHandler);
   }
 
-  void head(String path, Map<String, String> headers, Consumer<ErrorResponse> errorHandler);
+  void head(String path, Map<String, String> headers, ErrorHandler errorHandler);
 
   default <T extends RESTResponse> T delete(
       String path,
       Class<T> responseType,
       Supplier<Map<String, String>> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return delete(path, responseType, headers.get(), errorHandler);
   }
 
   <T extends RESTResponse> T delete(
-      String path,
-      Class<T> responseType,
-      Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler);
+      String path, Class<T> responseType, Map<String, String> headers, ErrorHandler errorHandler);
 
   default <T extends RESTResponse> T get(
       String path,
       Class<T> responseType,
       Supplier<Map<String, String>> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return get(path, ImmutableMap.of(), responseType, headers, errorHandler);
   }
 
   default <T extends RESTResponse> T get(
-      String path,
-      Class<T> responseType,
-      Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      String path, Class<T> responseType, Map<String, String> headers, ErrorHandler errorHandler) {
     return get(path, ImmutableMap.of(), responseType, headers, errorHandler);
   }
 
@@ -70,7 +61,7 @@ public interface RESTClient extends Closeable {
       Map<String, String> queryParams,
       Class<T> responseType,
       Supplier<Map<String, String>> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return get(path, queryParams, responseType, headers.get(), errorHandler);
   }
 
@@ -79,14 +70,14 @@ public interface RESTClient extends Closeable {
       Map<String, String> queryParams,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler);
+      ErrorHandler errorHandler);
 
   default <T extends RESTResponse> T post(
       String path,
       RESTRequest body,
       Class<T> responseType,
       Supplier<Map<String, String>> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return post(path, body, responseType, headers.get(), errorHandler);
   }
 
@@ -95,14 +86,14 @@ public interface RESTClient extends Closeable {
       RESTRequest body,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler);
+      ErrorHandler errorHandler);
 
   default <T extends RESTResponse> T postForm(
       String path,
       Map<String, String> formData,
       Class<T> responseType,
       Supplier<Map<String, String>> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return postForm(path, formData, responseType, headers.get(), errorHandler);
   }
 
@@ -111,5 +102,5 @@ public interface RESTClient extends Closeable {
       Map<String, String> formData,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler);
+      ErrorHandler errorHandler);
 }
diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTErrorResponse.java b/core/src/main/java/org/apache/iceberg/rest/RESTErrorResponse.java
new file mode 100644
index 0000000000..510dac0dae
--- /dev/null
+++ b/core/src/main/java/org/apache/iceberg/rest/RESTErrorResponse.java
@@ -0,0 +1,22 @@
+/*
+ * 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.iceberg.rest;
+
+/** Interface to mark a REST error response */
+public interface RESTErrorResponse extends RESTResponse {}
diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java
index 6dfb8756f3..613eef942b 100644
--- a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java
+++ b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java
@@ -46,6 +46,8 @@ import org.apache.iceberg.rest.requests.UpdateRequirementParser;
 import org.apache.iceberg.rest.requests.UpdateTableRequest.UpdateRequirement;
 import org.apache.iceberg.rest.responses.ErrorResponse;
 import org.apache.iceberg.rest.responses.ErrorResponseParser;
+import org.apache.iceberg.rest.responses.OAuthErrorResponse;
+import org.apache.iceberg.rest.responses.OAuthErrorResponseParser;
 import org.apache.iceberg.rest.responses.OAuthTokenResponse;
 import org.apache.iceberg.util.JsonUtil;
 
@@ -58,6 +60,8 @@ public class RESTSerializers {
     module
         .addSerializer(ErrorResponse.class, new ErrorResponseSerializer())
         .addDeserializer(ErrorResponse.class, new ErrorResponseDeserializer())
+        .addSerializer(OAuthErrorResponse.class, new OAuthErrorResponseSerializer())
+        .addDeserializer(OAuthErrorResponse.class, new OAuthErrorResponseDeserializer())
         .addSerializer(TableIdentifier.class, new TableIdentifierSerializer())
         .addDeserializer(TableIdentifier.class, new TableIdentifierDeserializer())
         .addSerializer(Namespace.class, new NamespaceSerializer())
@@ -149,6 +153,24 @@ public class RESTSerializers {
     }
   }
 
+  public static class OAuthErrorResponseDeserializer extends JsonDeserializer<OAuthErrorResponse> {
+    @Override
+    public OAuthErrorResponse deserialize(JsonParser p, DeserializationContext context)
+        throws IOException {
+      JsonNode node = p.getCodec().readTree(p);
+      return OAuthErrorResponseParser.fromJson(node);
+    }
+  }
+
+  public static class OAuthErrorResponseSerializer extends JsonSerializer<OAuthErrorResponse> {
+    @Override
+    public void serialize(
+        OAuthErrorResponse errorResponse, JsonGenerator gen, SerializerProvider serializers)
+        throws IOException {
+      OAuthErrorResponseParser.toJson(errorResponse, gen);
+    }
+  }
+
   public static class NamespaceDeserializer extends JsonDeserializer<Namespace> {
     @Override
     public Namespace deserialize(JsonParser p, DeserializationContext ctxt) throws IOException {
diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java
index 639856eb2d..02cdf82bc6 100644
--- a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java
+++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java
@@ -21,7 +21,6 @@ package org.apache.iceberg.rest;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.function.Consumer;
 import java.util.function.Supplier;
 import org.apache.iceberg.LocationProviders;
 import org.apache.iceberg.MetadataUpdate;
@@ -35,7 +34,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 import org.apache.iceberg.rest.requests.UpdateTableRequest;
-import org.apache.iceberg.rest.responses.ErrorResponse;
 import org.apache.iceberg.rest.responses.LoadTableResponse;
 
 class RESTTableOperations implements TableOperations {
@@ -102,7 +100,7 @@ class RESTTableOperations implements TableOperations {
   public void commit(TableMetadata base, TableMetadata metadata) {
     UpdateTableRequest.Builder requestBuilder;
     List<MetadataUpdate> baseChanges;
-    Consumer<ErrorResponse> errorHandler;
+    ErrorHandler errorHandler;
     switch (updateType) {
       case CREATE:
         Preconditions.checkState(
diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java
index edee9a6211..f16c4a39fa 100644
--- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java
+++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java
@@ -48,4 +48,12 @@ public class OAuth2Properties {
   public static final String SAML1_TOKEN_TYPE = "urn:ietf:params:oauth:token-type:saml1";
   public static final String SAML2_TOKEN_TYPE = "urn:ietf:params:oauth:token-type:saml2";
   public static final String JWT_TOKEN_TYPE = "urn:ietf:params:oauth:token-type:jwt";
+
+  // error type constants
+  public static final String INVALID_REQUEST_ERROR = "invalid_request";
+  public static final String INVALID_CLIENT_ERROR = "invalid_client";
+  public static final String INVALID_GRANT_ERROR = "invalid_grant";
+  public static final String UNAUTHORIZED_CLIENT_ERROR = "unauthorized_client";
+  public static final String UNSUPPORTED_GRANT_TYPE_ERROR = "unsupported_grant_type";
+  public static final String INVALID_SCOPE_ERROR = "invalid_scope";
 }
diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java
index 038037e2ce..8677fbdd53 100644
--- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java
+++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java
@@ -133,7 +133,7 @@ public class OAuth2Util {
             request,
             OAuthTokenResponse.class,
             headers,
-            ErrorHandlers.defaultErrorHandler());
+            ErrorHandlers.oauthErrorHandler());
     response.validate();
 
     return response;
@@ -161,7 +161,7 @@ public class OAuth2Util {
             request,
             OAuthTokenResponse.class,
             headers,
-            ErrorHandlers.defaultErrorHandler());
+            ErrorHandlers.oauthErrorHandler());
     response.validate();
 
     return response;
@@ -179,7 +179,7 @@ public class OAuth2Util {
             request,
             OAuthTokenResponse.class,
             headers,
-            ErrorHandlers.defaultErrorHandler());
+            ErrorHandlers.oauthErrorHandler());
     response.validate();
 
     return response;
diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java
index 5543259af1..037f92bc45 100644
--- a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java
+++ b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java
@@ -23,10 +23,10 @@ import java.io.StringWriter;
 import java.util.Arrays;
 import java.util.List;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
-import org.apache.iceberg.rest.RESTResponse;
+import org.apache.iceberg.rest.RESTErrorResponse;
 
 /** Standard response body for all API errors */
-public class ErrorResponse implements RESTResponse {
+public class ErrorResponse implements RESTErrorResponse {
 
   private String message;
   private String type;
diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponseParser.java
index a2973e05eb..326e6759ec 100644
--- a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponseParser.java
+++ b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponseParser.java
@@ -21,7 +21,6 @@ package org.apache.iceberg.rest.responses;
 import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.databind.JsonNode;
 import java.io.IOException;
-import java.io.StringWriter;
 import java.io.UncheckedIOException;
 import java.util.List;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
@@ -42,19 +41,7 @@ public class ErrorResponseParser {
   }
 
   public static String toJson(ErrorResponse errorResponse, boolean pretty) {
-    try {
-      StringWriter writer = new StringWriter();
-      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
-      if (pretty) {
-        generator.useDefaultPrettyPrinter();
-      }
-      toJson(errorResponse, generator);
-      generator.flush();
-      return writer.toString();
-    } catch (IOException e) {
-      throw new UncheckedIOException(
-          String.format("Failed to write error response json for: %s", errorResponse), e);
-    }
+    return JsonUtil.generate(gen -> toJson(errorResponse, gen), pretty);
   }
 
   public static void toJson(ErrorResponse errorResponse, JsonGenerator generator)
@@ -98,7 +85,6 @@ public class ErrorResponseParser {
         jsonNode != null && jsonNode.isObject(),
         "Cannot parse error response from non-object value: %s",
         jsonNode);
-    Preconditions.checkArgument(jsonNode.has(ERROR), "Cannot parse missing field: error");
     JsonNode error = JsonUtil.get(ERROR, jsonNode);
     String message = JsonUtil.getStringOrNull(MESSAGE, error);
     String type = JsonUtil.getStringOrNull(TYPE, error);
diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/OAuthErrorResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/OAuthErrorResponse.java
new file mode 100644
index 0000000000..35b0227d53
--- /dev/null
+++ b/core/src/main/java/org/apache/iceberg/rest/responses/OAuthErrorResponse.java
@@ -0,0 +1,100 @@
+/*
+ * 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.iceberg.rest.responses;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.rest.RESTErrorResponse;
+
+/** Standard response body for OAuth errors */
+public class OAuthErrorResponse implements RESTErrorResponse {
+
+  private String error;
+  private String errorDescription;
+  private String errorUri;
+
+  private OAuthErrorResponse(String error, String errorDescription, String errorUri) {
+    this.error = error;
+    this.errorDescription = errorDescription;
+    this.errorUri = errorUri;
+    validate();
+  }
+
+  @Override
+  public void validate() {
+    // Because we use the `OAuthErrorResponseParser`, validation is done there.
+  }
+
+  public String error() {
+    return error;
+  }
+
+  public String errorDescription() {
+    return errorDescription;
+  }
+
+  public String errorUri() {
+    return errorUri;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    return sb.append("OAuthErrorResponse(")
+        .append("error=")
+        .append(error)
+        .append(", errorDescription=")
+        .append(errorDescription)
+        .append(", errorUri=")
+        .append(errorUri)
+        .append(")")
+        .toString();
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static class Builder {
+    private String error;
+    private String errorDescription;
+    private String errorUri;
+
+    private Builder() {}
+
+    public Builder withError(String type) {
+      this.error = type;
+      return this;
+    }
+
+    public Builder withErrorDescription(String description) {
+      this.errorDescription = description;
+      return this;
+    }
+
+    public Builder withErrorUri(String uri) {
+      this.errorUri = uri;
+      return this;
+    }
+
+    public OAuthErrorResponse build() {
+      Preconditions.checkArgument(error != null, "Invalid response, missing field: error");
+      return new OAuthErrorResponse(error, errorDescription, errorUri);
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/OAuthErrorResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/OAuthErrorResponseParser.java
new file mode 100644
index 0000000000..90bed9848e
--- /dev/null
+++ b/core/src/main/java/org/apache/iceberg/rest/responses/OAuthErrorResponseParser.java
@@ -0,0 +1,83 @@
+/*
+ * 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.iceberg.rest.responses;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+
+public class OAuthErrorResponseParser {
+
+  private OAuthErrorResponseParser() {}
+
+  private static final String ERROR = "error";
+  private static final String ERROR_DESCRIPTION = "error_description";
+  private static final String ERROR_URI = "error_uri";
+
+  public static String toJson(OAuthErrorResponse errorResponse) {
+    return toJson(errorResponse, false);
+  }
+
+  public static String toJson(OAuthErrorResponse errorResponse, boolean pretty) {
+    return JsonUtil.generate(gen -> toJson(errorResponse, gen), pretty);
+  }
+
+  public static void toJson(OAuthErrorResponse errorResponse, JsonGenerator generator)
+      throws IOException {
+    generator.writeStartObject();
+
+    generator.writeStringField(ERROR, errorResponse.error());
+    generator.writeStringField(ERROR_DESCRIPTION, errorResponse.errorDescription());
+    generator.writeStringField(ERROR_URI, errorResponse.errorUri());
+
+    generator.writeEndObject();
+  }
+
+  /**
+   * Read OAuthErrorResponse from a JSON string.
+   *
+   * @param json a JSON string of an OAuthErrorResponse
+   * @return an OAuthErrorResponse object
+   */
+  public static OAuthErrorResponse fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to read JSON string: " + json, e);
+    }
+  }
+
+  public static OAuthErrorResponse fromJson(JsonNode jsonNode) {
+    Preconditions.checkArgument(
+        jsonNode != null && jsonNode.isObject(),
+        "Cannot parse error response from non-object value: %s",
+        jsonNode);
+    String error = JsonUtil.getString(ERROR, jsonNode);
+    String errorDescription = JsonUtil.getStringOrNull(ERROR_DESCRIPTION, jsonNode);
+    String errorUri = JsonUtil.getStringOrNull(ERROR_URI, jsonNode);
+    return OAuthErrorResponse.builder()
+        .withError(error)
+        .withErrorDescription(errorDescription)
+        .withErrorUri(errorUri)
+        .build();
+  }
+}
diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java
index d5af4d913b..843a23752f 100644
--- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java
+++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java
@@ -21,7 +21,6 @@ package org.apache.iceberg.rest;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import java.util.function.Consumer;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.SupportsNamespaces;
@@ -307,8 +306,10 @@ public class RESTCatalogAdapter implements RESTClient {
       Object body,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
+    // Note: we only handle catalog API errors as OAuth calls are mocked
     ErrorResponse.Builder errorBuilder = ErrorResponse.builder();
+
     Pair<Route, Map<String, String>> routeAndVars = Route.from(method, path);
     if (routeAndVars != null) {
       try {
@@ -332,7 +333,7 @@ public class RESTCatalogAdapter implements RESTClient {
     }
 
     ErrorResponse error = errorBuilder.build();
-    errorHandler.accept(error);
+    errorHandler.handle(error);
 
     // if the error handler doesn't throw an exception, throw a generic one
     throw new RESTException("Unhandled error: %s", error);
@@ -340,10 +341,7 @@ public class RESTCatalogAdapter implements RESTClient {
 
   @Override
   public <T extends RESTResponse> T delete(
-      String path,
-      Class<T> responseType,
-      Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      String path, Class<T> responseType, Map<String, String> headers, ErrorHandler errorHandler) {
     return execute(HTTPMethod.DELETE, path, null, null, responseType, headers, errorHandler);
   }
 
@@ -353,7 +351,7 @@ public class RESTCatalogAdapter implements RESTClient {
       RESTRequest body,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return execute(HTTPMethod.POST, path, null, body, responseType, headers, errorHandler);
   }
 
@@ -363,12 +361,12 @@ public class RESTCatalogAdapter implements RESTClient {
       Map<String, String> queryParams,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return execute(HTTPMethod.GET, path, queryParams, null, responseType, headers, errorHandler);
   }
 
   @Override
-  public void head(String path, Map<String, String> headers, Consumer<ErrorResponse> errorHandler) {
+  public void head(String path, Map<String, String> headers, ErrorHandler errorHandler) {
     execute(HTTPMethod.HEAD, path, null, null, null, headers, errorHandler);
   }
 
@@ -378,7 +376,7 @@ public class RESTCatalogAdapter implements RESTClient {
       Map<String, String> formData,
       Class<T> responseType,
       Map<String, String> headers,
-      Consumer<ErrorResponse> errorHandler) {
+      ErrorHandler errorHandler) {
     return execute(HTTPMethod.POST, path, null, formData, responseType, headers, errorHandler);
   }
 
diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java
index 6cf9fed73f..f3b251deb6 100644
--- a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java
+++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java
@@ -18,6 +18,11 @@
  */
 package org.apache.iceberg.rest;
 
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
 import static org.mockserver.integration.ClientAndServer.startClientAndServer;
 import static org.mockserver.model.HttpRequest.request;
 import static org.mockserver.model.HttpResponse.response;
@@ -28,8 +33,6 @@ import java.io.IOException;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Consumer;
 import org.apache.iceberg.AssertHelpers;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.IcebergBuild;
@@ -117,12 +120,9 @@ public class TestHTTPClient {
   public static void testHttpMethodOnSuccess(HttpMethod method) throws JsonProcessingException {
     Item body = new Item(0L, "hank");
     int statusCode = 200;
-    AtomicInteger errorCounter = new AtomicInteger(0);
-    Consumer<ErrorResponse> onError =
-        (error) -> {
-          errorCounter.incrementAndGet();
-          throw new RuntimeException("Failure response");
-        };
+
+    ErrorHandler onError = mock(ErrorHandler.class);
+    doThrow(new RuntimeException("Failure response")).when(onError).handle(any());
 
     String path = addRequestTestCaseAndGetPath(method, body, statusCode);
 
@@ -134,23 +134,22 @@ public class TestHTTPClient {
           successResponse,
           body);
     }
-    Assert.assertEquals(
-        "On a successful " + method + ", the error handler should not be called",
-        0,
-        errorCounter.get());
+
+    verify(onError, never()).handle(any());
   }
 
   public static void testHttpMethodOnFailure(HttpMethod method) throws JsonProcessingException {
     Item body = new Item(0L, "hank");
     int statusCode = 404;
-    AtomicInteger errorCounter = new AtomicInteger(0);
-    Consumer<ErrorResponse> onError =
-        error -> {
-          errorCounter.incrementAndGet();
-          throw new RuntimeException(
-              String.format(
-                  "Called error handler for method %s due to status code: %d", method, statusCode));
-        };
+
+    ErrorHandler onError = mock(ErrorHandler.class);
+    doThrow(
+            new RuntimeException(
+                String.format(
+                    "Called error handler for method %s due to status code: %d",
+                    method, statusCode)))
+        .when(onError)
+        .handle(any());
 
     String path = addRequestTestCaseAndGetPath(method, body, statusCode);
 
@@ -161,10 +160,7 @@ public class TestHTTPClient {
             "Called error handler for method %s due to status code: %d", method, statusCode),
         () -> doExecuteRequest(method, path, body, onError));
 
-    Assert.assertEquals(
-        "On an unsuccessful " + method + ", the error handler should be called",
-        1,
-        errorCounter.get());
+    verify(onError).handle(any());
   }
 
   // Adds a request that the mock-server can match against, based on the method, path, body, and
@@ -214,7 +210,7 @@ public class TestHTTPClient {
   }
 
   private static Item doExecuteRequest(
-      HttpMethod method, String path, Item body, Consumer<ErrorResponse> onError) {
+      HttpMethod method, String path, Item body, ErrorHandler onError) {
     Map<String, String> headers = ImmutableMap.of("Authorization", "Bearer " + BEARER_AUTH_TOKEN);
     switch (method) {
       case POST:
diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
index 2159c1c4ad..1f8cc57d55 100644
--- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
+++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java
@@ -30,7 +30,6 @@ import java.io.IOException;
 import java.nio.file.Path;
 import java.util.Map;
 import java.util.UUID;
-import java.util.function.Consumer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.iceberg.AssertHelpers;
 import org.apache.iceberg.CatalogProperties;
@@ -43,7 +42,6 @@ import org.apache.iceberg.jdbc.JdbcCatalog;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod;
 import org.apache.iceberg.rest.responses.ConfigResponse;
-import org.apache.iceberg.rest.responses.ErrorResponse;
 import org.apache.iceberg.rest.responses.LoadTableResponse;
 import org.apache.iceberg.rest.responses.OAuthTokenResponse;
 import org.apache.iceberg.types.Types;
@@ -98,7 +96,7 @@ public class TestRESTCatalog extends CatalogTests<RESTCatalog> {
               Object body,
               Class<T> responseType,
               Map<String, String> headers,
-              Consumer<ErrorResponse> errorHandler) {
+              ErrorHandler errorHandler) {
             // this doesn't use a Mockito spy because this is used for catalog tests, which have
             // different method calls
             if (!"v1/oauth/tokens".equals(path)) {
@@ -193,7 +191,7 @@ public class TestRESTCatalog extends CatalogTests<RESTCatalog> {
               Map<String, String> queryParams,
               Class<T> responseType,
               Map<String, String> headers,
-              Consumer<ErrorResponse> errorHandler) {
+              ErrorHandler errorHandler) {
             if ("v1/config".equals(path)) {
               return castResponse(
                   responseType,
diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestErrorResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestErrorResponseParser.java
index 98b764b678..8232f0fe34 100644
--- a/core/src/test/java/org/apache/iceberg/rest/responses/TestErrorResponseParser.java
+++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestErrorResponseParser.java
@@ -20,6 +20,7 @@ package org.apache.iceberg.rest.responses;
 
 import java.util.Arrays;
 import java.util.List;
+import org.assertj.core.api.Assertions;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -123,9 +124,9 @@ public class TestErrorResponseParser {
   }
 
   public void assertEquals(ErrorResponse expected, ErrorResponse actual) {
-    Assert.assertEquals("Message should be equal", expected.message(), actual.message());
-    Assert.assertEquals("Type should be equal", expected.type(), actual.type());
-    Assert.assertEquals("Response code should be equal", expected.code(), actual.code());
-    Assert.assertEquals("Stack should be equal", expected.stack(), actual.stack());
+    Assertions.assertThat(actual.message()).isEqualTo(expected.message());
+    Assertions.assertThat(actual.type()).isEqualTo(expected.type());
+    Assertions.assertThat(actual.code()).isEqualTo(expected.code());
+    Assertions.assertThat(actual.stack()).isEqualTo(expected.stack());
   }
 }
diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java
new file mode 100644
index 0000000000..23e4515370
--- /dev/null
+++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java
@@ -0,0 +1,114 @@
+/*
+ * 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.iceberg.rest.responses;
+
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestOAuthErrorResponseParser {
+
+  @Test
+  public void testOAuthErrorResponseToJson() {
+    String error = OAuth2Properties.INVALID_CLIENT_ERROR;
+    String description = "Credentials given were invalid";
+    String uri = "http://iceberg.apache.org";
+    String json =
+        String.format(
+            "{\"error\":\"%s\",\"error_description\":\"%s\",\"error_uri\":\"%s\"}",
+            error, description, uri);
+    OAuthErrorResponse response =
+        OAuthErrorResponse.builder()
+            .withError(error)
+            .withErrorDescription(description)
+            .withErrorUri(uri)
+            .build();
+    Assert.assertEquals(
+        "Should be able to serialize an error response as json",
+        OAuthErrorResponseParser.toJson(response),
+        json);
+  }
+
+  @Test
+  public void testOAuthErrorResponseToJsonWithNulls() {
+    String error = OAuth2Properties.INVALID_CLIENT_ERROR;
+    String expected =
+        String.format("{\"error\":\"%s\",\"error_description\":null,\"error_uri\":null}", error);
+    OAuthErrorResponse response = OAuthErrorResponse.builder().withError(error).build();
+    Assert.assertEquals(
+        "Should be able to serialize an error response as json",
+        OAuthErrorResponseParser.toJson(response),
+        expected);
+  }
+
+  @Test
+  public void testOAuthErrorResponseBuilderMissingError() {
+    Assertions.assertThatThrownBy(() -> OAuthErrorResponse.builder().build())
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Invalid response, missing field: error");
+  }
+
+  @Test
+  public void testOAuthErrorResponseFromJson() {
+    String error = OAuth2Properties.INVALID_CLIENT_ERROR;
+    String description = "Credentials given were invalid";
+    String uri = "http://iceberg.apache.org";
+    String json =
+        String.format(
+            "{\"error\":\"%s\",\"error_description\":\"%s\",\"error_uri\":\"%s\"}",
+            error, description, uri);
+    OAuthErrorResponse expected =
+        OAuthErrorResponse.builder()
+            .withError(error)
+            .withErrorDescription(description)
+            .withErrorUri(uri)
+            .build();
+    assertEquals(expected, OAuthErrorResponseParser.fromJson(json));
+  }
+
+  @Test
+  public void testOAuthErrorResponseFromJsonWithNulls() {
+    String error = OAuth2Properties.INVALID_CLIENT_ERROR;
+    String json = String.format("{\"error\":\"%s\"}", error);
+    OAuthErrorResponse expected = OAuthErrorResponse.builder().withError(error).build();
+    assertEquals(expected, OAuthErrorResponseParser.fromJson(json));
+
+    // test with explicitly set nulls
+    json = String.format("{\"error\":\"%s\",\"error_description\":null,\"error_uri\":null}", error);
+    assertEquals(expected, OAuthErrorResponseParser.fromJson(json));
+  }
+
+  @Test
+  public void testOAuthErrorResponseFromJsonMissingError() {
+    String description = "Credentials given were invalid";
+    String uri = "http://iceberg.apache.org";
+    String json =
+        String.format("{\"error_description\":\"%s\",\"error_uri\":\"%s\"}", description, uri);
+    Assertions.assertThatThrownBy(() -> OAuthErrorResponseParser.fromJson(json))
+        .isInstanceOf(IllegalArgumentException.class)
+        .hasMessage("Cannot parse missing string: error");
+  }
+
+  public void assertEquals(OAuthErrorResponse expected, OAuthErrorResponse actual) {
+    Assertions.assertThat(actual.error()).isEqualTo(expected.error());
+    Assertions.assertThat(actual.errorDescription()).isEqualTo(expected.errorDescription());
+    Assertions.assertThat(actual.errorUri()).isEqualTo(expected.errorUri());
+  }
+}
diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml
index 2b1d101e60..ebb1eddff5 100644
--- a/open-api/rest-catalog-open-api.yaml
+++ b/open-api/rest-catalog-open-api.yaml
@@ -173,11 +173,7 @@ paths:
         400:
           $ref: '#/components/responses/OAuthErrorResponse'
         401:
-          $ref: '#/components/responses/UnauthorizedResponse'
-        503:
-          $ref: '#/components/responses/ServiceUnavailableResponse'
-        5XX:
-          $ref: '#/components/responses/ServerErrorResponse'
+          $ref: '#/components/responses/OAuthErrorResponse'
 
   /v1/{prefix}/namespaces:
     parameters: