You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by sc...@apache.org on 2018/12/14 17:30:56 UTC
[beam] branch master updated: [BEAM-6206] Add CustomHttpErrors a
tool to allow adding custom errors for specific failing http calls. Plus,
add a custom error message in BigQueryServicesImpl. (#7270)
This is an automated email from the ASF dual-hosted git repository.
scott pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new 54e2fc1 [BEAM-6206] Add CustomHttpErrors a tool to allow adding custom errors for specific failing http calls. Plus, add a custom error message in BigQueryServicesImpl. (#7270)
54e2fc1 is described below
commit 54e2fc12ad6c07c43782d03fd95241934b36bda6
Author: Alex Amato <aj...@google.com>
AuthorDate: Fri Dec 14 09:30:48 2018 -0800
[BEAM-6206] Add CustomHttpErrors a tool to allow adding custom errors for specific failing http calls. Plus, add a custom error message in BigQueryServicesImpl. (#7270)
---
.../org/apache/beam/sdk/util/CustomHttpErrors.java | 141 +++++++++++++++++++++
.../apache/beam/sdk/util/HttpCallCustomError.java | 25 ++++
.../org/apache/beam/sdk/util/HttpCallMatcher.java | 28 ++++
.../apache/beam/sdk/util/HttpRequestWrapper.java | 40 ++++++
.../apache/beam/sdk/util/HttpResponseWrapper.java | 38 ++++++
.../beam/sdk/util/RetryHttpRequestInitializer.java | 32 ++++-
.../apache/beam/sdk/util/CustomHttpErrorsTest.java | 128 +++++++++++++++++++
.../sdk/io/gcp/bigquery/BigQueryServicesImpl.java | 16 +++
8 files changed, 443 insertions(+), 5 deletions(-)
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/CustomHttpErrors.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/CustomHttpErrors.java
new file mode 100644
index 0000000..db46d98
--- /dev/null
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/CustomHttpErrors.java
@@ -0,0 +1,141 @@
+/*
+ * 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.beam.sdk.util;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An optional component to use with the {@code RetryHttpRequestInitializer} in order to provide
+ * custom errors for failing http calls. This class allows you to specify custom error messages
+ * which match specific error codes and containing strings in the URL. The first matcher to match
+ * the request and response will be used to provide the custom error.
+ *
+ * <p>The intended use case here is to examine one of the logs emitted by a failing call made by the
+ * RetryHttpRequestInitializer, and then adding a custom error message which matches the URL and
+ * code for it.
+ *
+ * <p>Usage: See more in CustomHttpErrorsTest.
+ *
+ * <pre>{@code
+ * CustomHttpErrors.Builder builder = new CustomHttpErrors.Builder();
+ * builder.addErrorForCodeAndUrlContains(403,"/tables?", "Custom Error Msg");
+ * CustomHttpErrors customErrors = builder.build();
+ *
+ *
+ * RetryHttpRequestInitializer initializer = ...
+ * initializer.setCustomErrors(customErrors);
+ * }</pre>
+ *
+ * <p>Suggestions for future enhancements to anyone upgrading this file:
+ *
+ * <ul>
+ * <li>This class is left open for extension, to allow different functions for HttpCallMatcher and
+ * HttpCallCustomError to match and log errors. For example, new functionality may include
+ * matching an error based on the HttpResponse body. Additionally, extracting and logging
+ * strings from the HttpResponse body may make useful functionality.
+ * <li>Add a methods to add custom errors based on inspecting the contents of the HttpRequest and
+ * HttpResponse
+ * <li>Be sure to update the HttpRequestWrapper and HttpResponseWrapper with any new getters that
+ * you may use. The wrappers were introduced to add a layer of indirection which could be
+ * mocked mocked out in tests. This was unfortunately needed because mockito cannot mock final
+ * classes and its non trivial to just construct HttpRequest and HttpResponse objects.
+ * <li>Making matchers composable with an AND operator may simplify enhancing this code, if
+ * several different matchers are used.
+ * </ul>
+ *
+ * <p>
+ */
+public class CustomHttpErrors {
+
+ /**
+ * A simple Tuple class for creating a list of HttpResponseMatcher and HttpResponseCustomError to
+ * print for the responses.
+ */
+ @AutoValue
+ public abstract static class MatcherAndError implements Serializable {
+ static MatcherAndError create(HttpCallMatcher matcher, HttpCallCustomError customError) {
+ return new AutoValue_CustomHttpErrors_MatcherAndError(matcher, customError);
+ }
+
+ public abstract HttpCallMatcher getMatcher();
+
+ public abstract HttpCallCustomError getCustomError();
+ }
+
+ /** A Builder which allows building immutable CustomHttpErrors object. */
+ public static class Builder {
+
+ private List<MatcherAndError> matchersAndLogs = new ArrayList<MatcherAndError>();
+
+ public CustomHttpErrors build() {
+ return new CustomHttpErrors(this.matchersAndLogs);
+ }
+
+ /** Adds a matcher to log the provided string if the error matches a particular status code. */
+ public void addErrorForCode(int statusCode, String errorMessage) {
+ HttpCallMatcher matcher = (req, resp) -> resp.getStatusCode() == statusCode;
+ this.matchersAndLogs.add(MatcherAndError.create(matcher, simpleErrorMessage(errorMessage)));
+ }
+
+ /**
+ * Adds a matcher to log the provided string if the error matches a particular status code and
+ * the url contains a certain string.
+ */
+ public void addErrorForCodeAndUrlContains(
+ int statusCode, String urlContains, String errorMessage) {
+ HttpCallMatcher matcher =
+ (request, response) -> {
+ if (response.getStatusCode() == statusCode
+ && request.getUrl().toString().contains(urlContains)) {
+ return true;
+ }
+ return false;
+ };
+ this.matchersAndLogs.add(MatcherAndError.create(matcher, simpleErrorMessage(errorMessage)));
+ }
+
+ private static HttpCallCustomError simpleErrorMessage(String errorMessage) {
+ return (request, response) -> {
+ return errorMessage;
+ };
+ }
+ }
+
+ // The list of HttpRequest/Response matchers and functions to generate error strings.
+ private List<MatcherAndError> matchersAndLogs = new ArrayList<MatcherAndError>();
+
+ private CustomHttpErrors(List<MatcherAndError> matchersAndLogs) {
+ // Deep copy the matchersAndLogs, which allows the builder to be reused.
+ for (MatcherAndError m : matchersAndLogs) {
+ this.matchersAndLogs.add(m);
+ }
+ }
+
+ /** @return The the first custom error for the failing request and response to match, or null. */
+ public String getCustomError(HttpRequestWrapper req, HttpResponseWrapper res) {
+ for (MatcherAndError m : matchersAndLogs) {
+ if (m.getMatcher().matchResponse(req, res)) {
+ return m.getCustomError().customError(req, res);
+ }
+ }
+ return null;
+ }
+}
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpCallCustomError.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpCallCustomError.java
new file mode 100644
index 0000000..cb95c82
--- /dev/null
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpCallCustomError.java
@@ -0,0 +1,25 @@
+/*
+ * 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.beam.sdk.util;
+
+/** Lambda interface for defining a custom error to log based on an http request and response. */
+interface HttpCallCustomError {
+
+ /** @return A string which represents a custom error to be logged for the request and response. */
+ String customError(HttpRequestWrapper request, HttpResponseWrapper response);
+}
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpCallMatcher.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpCallMatcher.java
new file mode 100644
index 0000000..2437d45
--- /dev/null
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpCallMatcher.java
@@ -0,0 +1,28 @@
+/*
+ * 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.beam.sdk.util;
+
+/**
+ * Lambda interface for inspecting an http request and response to match the failure and possibly
+ * generate a custom error message with more context.
+ */
+interface HttpCallMatcher {
+
+ /** @return true iff the request and response represent a matching http c\all. */
+ boolean matchResponse(HttpRequestWrapper req, HttpResponseWrapper response);
+}
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpRequestWrapper.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpRequestWrapper.java
new file mode 100644
index 0000000..068a594
--- /dev/null
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpRequestWrapper.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.beam.sdk.util;
+
+import com.google.api.client.http.GenericUrl;
+import com.google.api.client.http.HttpRequest;
+
+/**
+ * These wrapper classes are necessary allow mocking out the HttpRequest and HttpResponse, since
+ * they are final classes and mockito cannot mock them. Note: There is an experimental mockito
+ * feature, but it causes many issues and several tests fail when it is enabled.
+ * https://stackoverflow.com/questions/14292863/how-to-mock-a-final-class-with-mockito
+ */
+class HttpRequestWrapper {
+
+ private HttpRequest request;
+
+ public HttpRequestWrapper(HttpRequest request) {
+ this.request = request;
+ }
+
+ public GenericUrl getUrl() {
+ return request.getUrl();
+ }
+}
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpResponseWrapper.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpResponseWrapper.java
new file mode 100644
index 0000000..3fdf780
--- /dev/null
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/HttpResponseWrapper.java
@@ -0,0 +1,38 @@
+/*
+ * 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.beam.sdk.util;
+
+import com.google.api.client.http.HttpResponse;
+
+/**
+ * These wrapper classes are necessary allow mocking out the HttpRequest and HttpResponse, since
+ * they are final classes and mockito cannot mock them. Note: There is an experimental mockito
+ * feature, but it causes many issues and several tests fail when it is enabled.
+ * https://stackoverflow.com/questions/14292863/how-to-mock-a-final-class-with-mockito
+ */
+class HttpResponseWrapper {
+ private HttpResponse response;
+
+ public HttpResponseWrapper(HttpResponse response) {
+ this.response = response;
+ }
+
+ public int getStatusCode() {
+ return response.getStatusCode();
+ }
+}
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java
index 9298227..2df2e60 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java
@@ -70,16 +70,19 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer {
private final Set<Integer> ignoredResponseCodes;
private int ioExceptionRetries;
private int unsuccessfulResponseRetries;
+ @Nullable private CustomHttpErrors customHttpErrors;
private LoggingHttpBackOffHandler(
Sleeper sleeper,
BackOff ioExceptionBackOff,
BackOff unsucessfulResponseBackOff,
- Set<Integer> ignoredResponseCodes) {
+ Set<Integer> ignoredResponseCodes,
+ @Nullable CustomHttpErrors customHttpErrors) {
this.sleeper = sleeper;
this.ioExceptionBackOff = ioExceptionBackOff;
this.unsuccessfulResponseBackOff = unsucessfulResponseBackOff;
this.ignoredResponseCodes = ignoredResponseCodes;
+ this.customHttpErrors = customHttpErrors;
}
@Override
@@ -126,12 +129,22 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer {
response.getStatusCode(),
request.getUrl());
} else {
+
String message =
"Request failed with code {}, "
+ "performed {} retries due to IOExceptions, "
+ "performed {} retries due to unsuccessful status codes, "
+ "HTTP framework says request {} be retried, "
- + "(caller responsible for retrying): {}";
+ + "(caller responsible for retrying): {}. {}";
+ String customLogMessage = "";
+ if (customHttpErrors != null) {
+ String error =
+ customHttpErrors.getCustomError(
+ new HttpRequestWrapper(request), new HttpResponseWrapper(response));
+ if (error != null) {
+ customLogMessage = error;
+ }
+ }
if (ignoredResponseCodes.contains(response.getStatusCode())) {
// Log ignored response codes at a lower level
LOG.debug(
@@ -140,7 +153,8 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer {
ioExceptionRetries,
unsuccessfulResponseRetries,
supportsRetry ? "can" : "cannot",
- request.getUrl());
+ request.getUrl(),
+ customLogMessage);
} else {
LOG.warn(
message,
@@ -148,7 +162,8 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer {
ioExceptionRetries,
unsuccessfulResponseRetries,
supportsRetry ? "can" : "cannot",
- request.getUrl());
+ request.getUrl(),
+ customLogMessage);
}
}
return willRetry;
@@ -173,6 +188,8 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer {
private final HttpResponseInterceptor responseInterceptor; // response Interceptor to use
+ private CustomHttpErrors customHttpErrors = null;
+
private final NanoClock nanoClock; // used for testing
private final Sleeper sleeper; // used for testing
@@ -235,7 +252,8 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer {
// their default values).
new ExponentialBackOff.Builder().setNanoClock(nanoClock).setMultiplier(2).build(),
new ExponentialBackOff.Builder().setNanoClock(nanoClock).setMultiplier(2).build(),
- ignoredResponseCodes);
+ ignoredResponseCodes,
+ this.customHttpErrors);
request.setUnsuccessfulResponseHandler(loggingHttpBackOffHandler);
request.setIOExceptionHandler(loggingHttpBackOffHandler);
@@ -246,6 +264,10 @@ public class RetryHttpRequestInitializer implements HttpRequestInitializer {
}
}
+ public void setCustomErrors(CustomHttpErrors customErrors) {
+ this.customHttpErrors = customErrors;
+ }
+
public void setWriteTimeout(int writeTimeout) {
this.writeTimeout = writeTimeout;
}
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/CustomHttpErrorsTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/CustomHttpErrorsTest.java
new file mode 100644
index 0000000..ddb5a85
--- /dev/null
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/util/CustomHttpErrorsTest.java
@@ -0,0 +1,128 @@
+/*
+ * 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.beam.sdk.util;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertNull;
+import static org.mockito.BDDMockito.given;
+import static org.mockito.BDDMockito.mock;
+
+import com.google.api.client.http.GenericUrl;
+import com.google.api.client.json.Json;
+import com.google.api.client.testing.http.MockLowLevelHttpResponse;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.MockitoAnnotations;
+
+/** Tests for CustomHttpErrorsTest. */
+@RunWith(JUnit4.class)
+public class CustomHttpErrorsTest {
+
+ private static final String BQ_TABLES_LIST_URL =
+ ("http://www.googleapis.com/bigquery/v2/projects/"
+ + "myproject/datasets/mydataset/tables?maxResults=1000");
+
+ @Before
+ public void initMocks() {
+ MockitoAnnotations.initMocks(this);
+ }
+
+ private static MockLowLevelHttpResponse createResponse(int code, String body) {
+ MockLowLevelHttpResponse response = new MockLowLevelHttpResponse();
+ response.addHeader("custom_header", "value");
+ response.setStatusCode(code);
+ response.setContentType(Json.MEDIA_TYPE);
+ response.setContent(body);
+ return response;
+ }
+
+ private HttpRequestWrapper createHttpRequest(String url) throws MalformedURLException {
+ HttpRequestWrapper request = mock(HttpRequestWrapper.class);
+ GenericUrl genericUrl = new GenericUrl(new URL(url));
+ given(request.getUrl()).willReturn(genericUrl);
+ return request;
+ }
+
+ private HttpResponseWrapper createHttpResponse(int statusCode) {
+ HttpResponseWrapper response = mock(HttpResponseWrapper.class);
+ given(response.getStatusCode()).willReturn(statusCode);
+ return response;
+ }
+
+ @Test
+ public void testMatchesCode() throws IOException {
+ HttpRequestWrapper request = createHttpRequest(BQ_TABLES_LIST_URL);
+ HttpResponseWrapper response = createHttpResponse(403);
+ HttpCallCustomError mockCustomError = mock(HttpCallCustomError.class);
+
+ CustomHttpErrors.Builder builder = new CustomHttpErrors.Builder();
+ builder.addErrorForCode(403, "Custom Error Msg");
+ CustomHttpErrors customErrors = builder.build();
+
+ String errorMessage = customErrors.getCustomError(request, response);
+ assertEquals("Custom Error Msg", errorMessage);
+ }
+
+ @Test
+ public void testNotMatchesCode() throws IOException {
+ HttpRequestWrapper request = createHttpRequest(BQ_TABLES_LIST_URL);
+ HttpResponseWrapper response = createHttpResponse(404);
+ HttpCallCustomError mockCustomError = mock(HttpCallCustomError.class);
+
+ CustomHttpErrors.Builder builder = new CustomHttpErrors.Builder();
+ builder.addErrorForCode(403, "Custom Error Msg");
+
+ CustomHttpErrors customErrors = builder.build();
+
+ String errorMessage = customErrors.getCustomError(request, response);
+ assertNull(errorMessage);
+ }
+
+ @Test
+ public void testMatchesCodeAndUrlContains() throws IOException {
+ HttpRequestWrapper request = createHttpRequest(BQ_TABLES_LIST_URL);
+ HttpResponseWrapper response = createHttpResponse(403);
+ HttpCallCustomError mockCustomError = mock(HttpCallCustomError.class);
+
+ CustomHttpErrors.Builder builder = new CustomHttpErrors.Builder();
+ builder.addErrorForCodeAndUrlContains(403, "/tables?", "Custom Error Msg");
+ CustomHttpErrors customErrors = builder.build();
+
+ String errorMessage = customErrors.getCustomError(request, response);
+ assertEquals("Custom Error Msg", errorMessage);
+ }
+
+ @Test
+ public void testNotMatchesCodeAndUrlContains() throws IOException {
+ HttpRequestWrapper request = createHttpRequest(BQ_TABLES_LIST_URL);
+ HttpResponseWrapper response = createHttpResponse(404);
+ HttpCallCustomError mockCustomError = mock(HttpCallCustomError.class);
+
+ CustomHttpErrors.Builder builder = new CustomHttpErrors.Builder();
+ builder.addErrorForCodeAndUrlContains(403, "/doesnotmatch?", "Custom Error Msg");
+ CustomHttpErrors customErrors = builder.build();
+
+ String errorMessage = customErrors.getCustomError(request, response);
+ assertNull(errorMessage);
+ }
+}
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
index f147634..524eccb 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -63,6 +63,7 @@ import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.util.BackOffAdapter;
+import org.apache.beam.sdk.util.CustomHttpErrors;
import org.apache.beam.sdk.util.FluentBackoff;
import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
import org.apache.beam.sdk.util.Transport;
@@ -906,6 +907,7 @@ class BigQueryServicesImpl implements BigQueryServices {
private static Bigquery.Builder newBigQueryClient(BigQueryOptions options) {
RetryHttpRequestInitializer httpRequestInitializer =
new RetryHttpRequestInitializer(ImmutableList.of(404));
+ httpRequestInitializer.setCustomErrors(createBigQueryClientCustomErrors());
httpRequestInitializer.setWriteTimeout(options.getHTTPWriteTimeout());
return new Bigquery.Builder(
Transport.getTransport(),
@@ -928,4 +930,18 @@ class BigQueryServicesImpl implements BigQueryServices {
new HttpCredentialsAdapter(credential), httpRequestInitializer);
}
}
+
+ public static CustomHttpErrors createBigQueryClientCustomErrors() {
+ CustomHttpErrors.Builder builder = new CustomHttpErrors.Builder();
+ // 403 errors, to list tables, matching this URL:
+ // http://www.googleapis.com/bigquery/v2/projects/myproject/datasets/
+ // mydataset/tables?maxResults=1000
+ builder.addErrorForCodeAndUrlContains(
+ 403,
+ "/tables?",
+ "The GCP project is most likely exceeding the rate limit on "
+ + "bigquery.tables.list, please find the instructions to increase this limit at: "
+ + "https://cloud.google.com/service-infrastructure/docs/rate-limiting#configure");
+ return builder.build();
+ }
}