Skip to content

Commit

Permalink
REST: implement handling of OAuth error responses (apache#5698)
Browse files Browse the repository at this point in the history
* 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
  • Loading branch information
bryanck authored Sep 19, 2022
1 parent f8aecf6 commit c293af2
Show file tree
Hide file tree
Showing 19 changed files with 538 additions and 136 deletions.
26 changes: 26 additions & 0 deletions core/src/main/java/org/apache/iceberg/rest/ErrorHandler.java
Original file line number Diff line number Diff line change
@@ -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);
}
122 changes: 95 additions & 27 deletions core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -30,7 +29,11 @@
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
Expand All @@ -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());
Expand All @@ -64,15 +80,18 @@ private static Consumer<ErrorResponse> baseCommitErrorHandler() {
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())) {
Expand All @@ -83,15 +102,18 @@ private static Consumer<ErrorResponse> baseTableErrorHandler() {
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());
Expand All @@ -100,15 +122,26 @@ private static Consumer<ErrorResponse> baseNamespaceErrorHandler() {
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());
Expand All @@ -128,6 +161,41 @@ public static Consumer<ErrorResponse> defaultErrorHandler() {
}

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());
}
}
}
}
}
27 changes: 11 additions & 16 deletions core/src/main/java/org/apache/iceberg/rest/HTTPClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
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;
Expand All @@ -46,7 +45,6 @@
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;

Expand Down Expand Up @@ -88,7 +86,7 @@ private static boolean isSuccessful(CloseableHttpResponse response) {
|| 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()
Expand All @@ -105,12 +103,12 @@ private static ErrorResponse buildDefaultErrorResponse(CloseableHttpResponse res
// 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
Expand All @@ -129,7 +127,7 @@ private static void throwFailure(
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);
Expand Down Expand Up @@ -171,7 +169,7 @@ private <T> T execute(
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);
Expand Down Expand Up @@ -227,7 +225,7 @@ private <T> T execute(
}

@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);
}

Expand All @@ -237,7 +235,7 @@ public <T extends RESTResponse> T get(
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);
}

Expand All @@ -247,16 +245,13 @@ public <T extends RESTResponse> T post(
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);
}

Expand All @@ -266,7 +261,7 @@ public <T extends RESTResponse> T postForm(
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);
}

Expand Down
Loading

0 comments on commit c293af2

Please sign in to comment.