diff --git a/CHANGES.md b/CHANGES.md index ba020af3fa85..d3282e1c31cd 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -64,6 +64,7 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Upgraded GoogleAdsAPI to v19 for GoogleAdsIO (Java) ([#34497](https://github.com/apache/beam/pull/34497)). Changed PTransform method from version-specified (`v17()`) to `current()` for better backward compatibility in the future. ## New Features / Improvements diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c30c89f6bdd5..cea37b58abb8 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -726,8 +726,6 @@ class BeamModulePlugin implements Plugin { gax_grpc : "com.google.api:gax-grpc", // google_cloud_platform_libraries_bom sets version gax_grpc_test : "com.google.api:gax-grpc:$gax_version:testlib", // google_cloud_platform_libraries_bom sets version gax_httpjson : "com.google.api:gax-httpjson", // google_cloud_platform_libraries_bom sets version - google_ads : "com.google.api-ads:google-ads:$google_ads_version", - google_ads_stubs : "com.google.api-ads:google-ads-stubs-v17:$google_ads_version", google_api_client : "com.google.api-client:google-api-client:$google_clients_version", // for the libraries using $google_clients_version below. google_api_client_gson : "com.google.api-client:google-api-client-gson:$google_clients_version", google_api_client_java6 : "com.google.api-client:google-api-client-java6:$google_clients_version", diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml index 01c997441479..af384ff19c09 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml @@ -62,8 +62,7 @@ - - + diff --git a/sdks/java/io/google-ads/build.gradle b/sdks/java/io/google-ads/build.gradle index 037fcbf28a6a..0796c412a65f 100644 --- a/sdks/java/io/google-ads/build.gradle +++ b/sdks/java/io/google-ads/build.gradle @@ -23,6 +23,8 @@ description = "Apache Beam :: SDKs :: Java :: IO :: Google Ads" ext.summary = "IO to read from Google Ads" dependencies { + implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) + implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:extensions:google-cloud-platform-core") implementation library.java.jackson_annotations @@ -31,8 +33,8 @@ dependencies { implementation library.java.google_auth_library_oauth2_http implementation library.java.protobuf_java implementation library.java.protobuf_java_util - implementation library.java.google_ads - implementation library.java.google_ads_stubs + implementation "com.google.api-ads:google-ads:36.0.0" + implementation "com.google.api-ads:google-ads-stubs-v19:36.0.0" implementation library.java.joda_time implementation library.java.vendored_guava_32_1_2_jre testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") diff --git a/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsIO.java b/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsIO.java index c36e75667e7c..d90c7751e6b7 100644 --- a/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsIO.java +++ b/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsIO.java @@ -17,17 +17,84 @@ */ package org.apache.beam.sdk.io.googleads; +import com.google.protobuf.Message; +import java.io.Serializable; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.checkerframework.checker.nullness.qual.Nullable; + /** * {@link GoogleAdsIO} provides an API for reading from the Google Ads API over supported * versions of the Google Ads client libraries. * - * @see GoogleAdsV17 + * @see GoogleAdsV19 */ -public class GoogleAdsIO { - private GoogleAdsIO() {} +public abstract class GoogleAdsIO { + + @SuppressWarnings( + "TypeParameterUnusedInFormals") // for source code backward compatible when underlying API + // version changed + public abstract , PCollection>> T read(); + + @SuppressWarnings( + "TypeParameterUnusedInFormals") // for source code backward compatible when underlying API + // version changed + public abstract < + T extends + PTransform, PCollection>> + T readAll(); + + public static GoogleAdsV19 current() { + return GoogleAdsV19.INSTANCE; + } + + /** + * Implement this interface to create a {@link RateLimitPolicy}. This should be used to limit all + * traffic sent to the Google Ads API for a pair of developer token and customer ID and any other + * relevant attributes for the specific Google Ads API service being called. + */ + public interface RateLimitPolicyFactory extends Serializable { + RateLimitPolicy getRateLimitPolicy(); + } + + /** + * This interface can be used to implement custom client-side rate limiting policies. Custom + * policies should follow best practices for interacting with the Google Ads API. + * + * @see Best + * Practices in the Google Ads documentation + */ + public interface RateLimitPolicy { + /** + * Called before a request is sent. + * + * @param developerToken The developer token used for the request. + * @param customerId The customer ID specified on the request. + * @param request Any Google Ads API request. + * @throws InterruptedException + */ + void onBeforeRequest(@Nullable String developerToken, String customerId, Message request) + throws InterruptedException; + + /** + * Called after a request succeeds. + * + * @param developerToken The developer token used for the request. + * @param customerId The customer ID specified on the request. + * @param request Any Google Ads API request. + */ + void onSuccess(@Nullable String developerToken, String customerId, Message request); - public static GoogleAdsV17 v17() { - return GoogleAdsV17.INSTANCE; + /** + * Called after a request fails with a retryable error. + * + * @param developerToken The developer token used for the request. + * @param customerId The customer ID specified on the request. + * @param request Any Google Ads API request. + * @param error A retryable error. + */ + void onError( + @Nullable String developerToken, String customerId, Message request, GoogleAdsErrorT error); } } diff --git a/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV17.java b/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV19.java similarity index 83% rename from sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV17.java rename to sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV19.java index df6c09b5f705..11f11ea5f8c7 100644 --- a/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV17.java +++ b/sdks/java/io/google-ads/src/main/java/org/apache/beam/sdk/io/googleads/GoogleAdsV19.java @@ -22,20 +22,19 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import com.google.ads.googleads.lib.GoogleAdsClient; -import com.google.ads.googleads.v17.errors.GoogleAdsError; -import com.google.ads.googleads.v17.errors.GoogleAdsException; -import com.google.ads.googleads.v17.errors.GoogleAdsFailure; -import com.google.ads.googleads.v17.errors.InternalErrorEnum; -import com.google.ads.googleads.v17.errors.QuotaErrorEnum; -import com.google.ads.googleads.v17.services.GoogleAdsRow; -import com.google.ads.googleads.v17.services.GoogleAdsServiceClient; -import com.google.ads.googleads.v17.services.SearchGoogleAdsStreamRequest; -import com.google.ads.googleads.v17.services.SearchGoogleAdsStreamResponse; +import com.google.ads.googleads.v19.errors.GoogleAdsError; +import com.google.ads.googleads.v19.errors.GoogleAdsException; +import com.google.ads.googleads.v19.errors.GoogleAdsFailure; +import com.google.ads.googleads.v19.errors.InternalErrorEnum; +import com.google.ads.googleads.v19.errors.QuotaErrorEnum; +import com.google.ads.googleads.v19.services.GoogleAdsRow; +import com.google.ads.googleads.v19.services.GoogleAdsServiceClient; +import com.google.ads.googleads.v19.services.SearchGoogleAdsStreamRequest; +import com.google.ads.googleads.v19.services.SearchGoogleAdsStreamResponse; import com.google.auto.value.AutoValue; import com.google.protobuf.Message; import com.google.protobuf.util.Durations; import java.io.IOException; -import java.io.Serializable; import java.util.Optional; import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.options.PipelineOptions; @@ -58,7 +57,10 @@ import org.joda.time.Duration; /** - * {@link GoogleAdsV17} provides an API to read Google Ads API v17 reports. + * {@link GoogleAdsV19} provides an API to read Google Ads API v19 reports. + * + *

No backward compatibility guaranteed. Do not use directly. Use {@link GoogleAdsIO#current()} + * to access GoogleAdsIO instead. * *

The Google Ads API does not use service account credentials in the same way as Google Cloud * Platform APIs do. Service account credentials are typically only used to delegate (using @@ -77,15 +79,15 @@ * --googleAdsDeveloperToken=your-developer-token * * - *

Use {@link GoogleAdsV17#read()} to read either a bounded or unbounded {@link PCollection} of + *

Use {@link GoogleAdsV19#read()} to read either a bounded or unbounded {@link PCollection} of * {@link GoogleAdsRow} from a single Google Ads Query * Language query using {@link Read#withQuery(String)} and a {@link PCollection} of customer - * IDs. Alternatively, use {@link GoogleAdsV17#readAll()} to read either a bounded or unbounded + * IDs. Alternatively, use {@link GoogleAdsV19#readAll()} to read either a bounded or unbounded * {@link PCollection} of {@link GoogleAdsRow} from a {@link PCollection} of {@link * SearchGoogleAdsStreamRequest} potentially containing many different queries. * - *

For example, using {@link GoogleAdsV17#read()}: + *

For example, using {@link GoogleAdsV19#read()}: * *

{@code
  * Pipeline p = Pipeline.create();
@@ -93,7 +95,7 @@
  *     p.apply(Create.of(Long.toString(1234567890L)));
  * PCollection rows =
  *     customerIds.apply(
- *         GoogleAdsIO.v17()
+ *         GoogleAdsIO.current()
  *             .read()
  *             .withRateLimitPolicy(MY_RATE_LIMIT_POLICY)
  *             .withQuery(
@@ -105,7 +107,7 @@
  * p.run();
  * }
* - *

Alternatively, using {@link GoogleAdsV17#readAll()} to execute requests from a {@link + *

Alternatively, using {@link GoogleAdsV19#readAll()} to execute requests from a {@link * PCollection} of {@link SearchGoogleAdsStreamRequest}: * *

{@code
@@ -124,13 +126,13 @@
  *                             + "FROM campaign")
  *                     .build())));
  * PCollection rows =
- *     requests.apply(GoogleAdsIO.v17().readAll().withRateLimitPolicy(MY_RATE_LIMIT_POLICY));
+ *     requests.apply(GoogleAdsIO.current().readAll().withRateLimitPolicy(MY_RATE_LIMIT_POLICY));
  * p.run();
  * }
* *

Client-side rate limiting

* - * On construction of a {@link GoogleAdsV17#read()} or {@link GoogleAdsV17#readAll()} transform a + * On construction of a {@link GoogleAdsV19#read()} or {@link GoogleAdsV19#readAll()} transform a * rate limiting policy must be specified to stay well under the assigned quota for the Google Ads * API. The Google Ads API enforces global rate limits from the developer token down to the customer * ID and depending on the access level of the developer token a limit on the total number of @@ -154,24 +156,26 @@ * Functionality and Rate * sheet & non-compliance fees in the Google Ads API documentation for more details. * - * @see GoogleAdsIO#v17() + * @see GoogleAdsIO#current() * @see GoogleAdsOptions * @see Best * Practices in the Google Ads documentation */ -public class GoogleAdsV17 { - static final GoogleAdsV17 INSTANCE = new GoogleAdsV17(); +public class GoogleAdsV19 extends GoogleAdsIO { + static final GoogleAdsV19 INSTANCE = new GoogleAdsV19(); - private GoogleAdsV17() {} + private GoogleAdsV19() {} + @Override public Read read() { - return new AutoValue_GoogleAdsV17_Read.Builder() + return new AutoValue_GoogleAdsV19_Read.Builder() .setGoogleAdsClientFactory(DefaultGoogleAdsClientFactory.getInstance()) .build(); } + @Override public ReadAll readAll() { - return new AutoValue_GoogleAdsV17_ReadAll.Builder() + return new AutoValue_GoogleAdsV19_ReadAll.Builder() .setGoogleAdsClientFactory(DefaultGoogleAdsClientFactory.getInstance()) .build(); } @@ -180,7 +184,7 @@ public ReadAll readAll() { * A {@link PTransform} that reads the results of a Google Ads query as {@link GoogleAdsRow} * objects. * - * @see GoogleAdsIO#v17() + * @see GoogleAdsIO#current() * @see #readAll() */ @AutoValue @@ -194,7 +198,7 @@ public abstract static class Read abstract GoogleAdsClientFactory getGoogleAdsClientFactory(); - abstract @Nullable RateLimitPolicyFactory getRateLimitPolicyFactory(); + abstract @Nullable RateLimitPolicyFactory getRateLimitPolicyFactory(); abstract Builder toBuilder(); @@ -208,7 +212,8 @@ abstract static class Builder { abstract Builder setGoogleAdsClientFactory(GoogleAdsClientFactory googleAdsClientFactory); - abstract Builder setRateLimitPolicyFactory(RateLimitPolicyFactory rateLimitPolicyFactory); + abstract Builder setRateLimitPolicyFactory( + RateLimitPolicyFactory rateLimitPolicyFactory); abstract Read build(); } @@ -282,7 +287,7 @@ public Read withGoogleAdsClientFactory(GoogleAdsClientFactory googleAdsClientFac * @return A new {@link Read} transform with the specified rate limit policy factory. * @see GoogleAdsClient */ - public Read withRateLimitPolicy(RateLimitPolicyFactory rateLimitPolicyFactory) { + public Read withRateLimitPolicy(RateLimitPolicyFactory rateLimitPolicyFactory) { checkArgumentNotNull(rateLimitPolicyFactory, "rateLimitPolicyFactory cannot be null"); return toBuilder().setRateLimitPolicyFactory(rateLimitPolicyFactory).build(); @@ -291,7 +296,7 @@ public Read withRateLimitPolicy(RateLimitPolicyFactory rateLimitPolicyFactory) { @Override public PCollection expand(PCollection input) { String query = getQuery(); - RateLimitPolicyFactory rateLimitPolicyFactory = getRateLimitPolicyFactory(); + RateLimitPolicyFactory rateLimitPolicyFactory = getRateLimitPolicyFactory(); checkArgumentNotNull(query, "withQuery() is required"); checkArgumentNotNull(rateLimitPolicyFactory, "withRateLimitPolicy() is required"); @@ -325,7 +330,7 @@ public void populateDisplayData(DisplayData.Builder builder) { * A {@link PTransform} that reads the results of many {@link SearchGoogleAdsStreamRequest} * objects as {@link GoogleAdsRow} objects. * * - * @see GoogleAdsIO#v17() + * @see GoogleAdsIO#current() * @see #readAll() */ @AutoValue @@ -337,7 +342,7 @@ public abstract static class ReadAll abstract GoogleAdsClientFactory getGoogleAdsClientFactory(); - abstract @Nullable RateLimitPolicyFactory getRateLimitPolicyFactory(); + abstract @Nullable RateLimitPolicyFactory getRateLimitPolicyFactory(); abstract Builder toBuilder(); @@ -349,7 +354,8 @@ abstract static class Builder { abstract Builder setGoogleAdsClientFactory(GoogleAdsClientFactory googleAdsClientFactory); - abstract Builder setRateLimitPolicyFactory(RateLimitPolicyFactory rateLimitPolicyFactory); + abstract Builder setRateLimitPolicyFactory( + RateLimitPolicyFactory rateLimitPolicyFactory); abstract ReadAll build(); } @@ -408,7 +414,8 @@ public ReadAll withGoogleAdsClientFactory(GoogleAdsClientFactory googleAdsClient * @return A new {@link ReadAll} transform with the specified rate limit policy factory. * @see GoogleAdsClient */ - public ReadAll withRateLimitPolicy(RateLimitPolicyFactory rateLimitPolicyFactory) { + public ReadAll withRateLimitPolicy( + RateLimitPolicyFactory rateLimitPolicyFactory) { checkArgumentNotNull(rateLimitPolicyFactory, "rateLimitPolicyFactory cannot be null"); return toBuilder().setRateLimitPolicyFactory(rateLimitPolicyFactory).build(); @@ -443,13 +450,13 @@ static class ReadAllFn extends DoFn @VisibleForTesting static Sleeper sleeper = Sleeper.DEFAULT; - private final GoogleAdsV17.ReadAll spec; + private final GoogleAdsV19.ReadAll spec; private transient @Nullable GoogleAdsClient googleAdsClient; private transient @Nullable GoogleAdsServiceClient googleAdsServiceClient; - private transient @Nullable RateLimitPolicy rateLimitPolicy; + private transient @Nullable RateLimitPolicy rateLimitPolicy; - ReadAllFn(GoogleAdsV17.ReadAll spec) { + ReadAllFn(GoogleAdsV19.ReadAll spec) { this.spec = spec; } @@ -463,8 +470,8 @@ public void setup(PipelineOptions options) { .newGoogleAdsClient( adsOptions, spec.getDeveloperToken(), null, spec.getLoginCustomerId()); final GoogleAdsServiceClient googleAdsServiceClient = - googleAdsClient.getVersion17().createGoogleAdsServiceClient(); - final RateLimitPolicy rateLimitPolicy = + googleAdsClient.getVersion19().createGoogleAdsServiceClient(); + final RateLimitPolicy rateLimitPolicy = checkStateNotNull(spec.getRateLimitPolicyFactory()).getRateLimitPolicy(); this.googleAdsClient = googleAdsClient; @@ -477,7 +484,7 @@ public void setup(PipelineOptions options) { public void processElement(ProcessContext c) throws IOException, InterruptedException { final GoogleAdsClient googleAdsClient = this.googleAdsClient; final GoogleAdsServiceClient googleAdsServiceClient = this.googleAdsServiceClient; - final RateLimitPolicy rateLimitPolicy = this.rateLimitPolicy; + final RateLimitPolicy rateLimitPolicy = this.rateLimitPolicy; BackOff backoff = BACKOFF.backoff(); BackOff nextBackoff = backoff; @@ -565,81 +572,32 @@ private Optional findFirstRetryableError(GoogleAdsFailure e) { } } - /** - * Implement this interface to create a {@link RateLimitPolicy}. This should be used to limit all - * traffic sent to the Google Ads API for a pair of developer token and customer ID and any other - * relevant attributes for the specific Google Ads API service being called. - */ - public interface RateLimitPolicyFactory extends Serializable { - RateLimitPolicy getRateLimitPolicy(); - } - - /** - * This interface can be used to implement custom client-side rate limiting policies. Custom - * policies should follow best practices for interacting with the Google Ads API. - * - * @see Best - * Practices in the Google Ads documentation - */ - public interface RateLimitPolicy { - /** - * Called before a request is sent. - * - * @param developerToken The developer token used for the request. - * @param customerId The customer ID specified on the request. - * @param request Any Google Ads API request. - * @throws InterruptedException - */ - void onBeforeRequest(@Nullable String developerToken, String customerId, Message request) - throws InterruptedException; - - /** - * Called after a request succeeds. - * - * @param developerToken The developer token used for the request. - * @param customerId The customer ID specified on the request. - * @param request Any Google Ads API request. - */ - void onSuccess(@Nullable String developerToken, String customerId, Message request); - - /** - * Called after a request fails with a retryable error. - * - * @param developerToken The developer token used for the request. - * @param customerId The customer ID specified on the request. - * @param request Any Google Ads API request. - * @param error A retryable error. - */ - void onError( - @Nullable String developerToken, String customerId, Message request, GoogleAdsError error); - } - /** * This rate limit policy wraps a {@link RateLimiter} and can be used in low volume and * development use cases as a client-side rate limiting policy. This policy does not enforce a * global (per pipeline or otherwise) rate limit to requests and should not be used in deployments * where the Google Ads API quota is shared between multiple applications. * - *

This policy can be used to limit requests across all {@link GoogleAdsV17.Read} or {@link - * GoogleAdsV17.ReadAll} transforms by defining and using a {@link - * GoogleAdsV17.RateLimitPolicyFactory} which holds a shared static {@link - * GoogleAdsV17.SimpleRateLimitPolicy}. Note that the desired rate must be divided by the expected + *

This policy can be used to limit requests across all {@link GoogleAdsV19.Read} or {@link + * GoogleAdsV19.ReadAll} transforms by defining and using a {@link + * GoogleAdsV19.RateLimitPolicyFactory} which holds a shared static {@link + * GoogleAdsV19.SimpleRateLimitPolicy}. Note that the desired rate must be divided by the expected * maximum number of workers for the pipeline, otherwise the pipeline may exceed the desired rate * after an upscaling event. * *

{@code
-   * public class SimpleRateLimitPolicyFactory implements GoogleAdsV17.RateLimitPolicyFactory {
-   *   private static final GoogleAdsV17.RateLimitPolicy POLICY =
-   *       new GoogleAdsV17.SimpleRateLimitPolicy(1.0 / 1000.0);
+   * public class SimpleRateLimitPolicyFactory implements GoogleAdsIO.RateLimitPolicyFactory {
+   *   private static final GoogleAdsIO.RateLimitPolicy POLICY =
+   *       new GoogleAdsV19.SimpleRateLimitPolicy(1.0 / 1000.0);
    *
    *   @Override
-   *   public GoogleAdsV17.RateLimitPolicy getRateLimitPolicy() {
+   *   public GoogleAdsIO.RateLimitPolicy getRateLimitPolicy() {
    *     return POLICY;
    *   }
    * }
    * }
*/ - public static class SimpleRateLimitPolicy implements RateLimitPolicy { + public static class SimpleRateLimitPolicy implements RateLimitPolicy { private final RateLimiter rateLimiter; public SimpleRateLimitPolicy(double permitsPerSecond) { diff --git a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/DummyRateLimitPolicy.java b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/DummyRateLimitPolicy.java index b958f28f83b0..56ca204db4f4 100644 --- a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/DummyRateLimitPolicy.java +++ b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/DummyRateLimitPolicy.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.io.googleads; -import com.google.ads.googleads.v17.errors.GoogleAdsError; +import com.google.ads.googleads.v19.errors.GoogleAdsError; import com.google.protobuf.Message; import org.checkerframework.checker.nullness.qual.Nullable; -public class DummyRateLimitPolicy implements GoogleAdsV17.RateLimitPolicy { +public class DummyRateLimitPolicy implements GoogleAdsV19.RateLimitPolicy { @Override public void onBeforeRequest(@Nullable String developerToken, String customerId, Message request) throws InterruptedException {} diff --git a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsV17Test.java b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsIOTest.java similarity index 87% rename from sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsV17Test.java rename to sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsIOTest.java index f73ab10e0824..4804918bed6c 100644 --- a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsV17Test.java +++ b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsIOTest.java @@ -20,18 +20,18 @@ import static org.mockito.Mockito.any; import static org.mockito.Mockito.when; -import com.google.ads.googleads.v17.errors.AuthenticationErrorEnum.AuthenticationError; -import com.google.ads.googleads.v17.errors.ErrorCode; -import com.google.ads.googleads.v17.errors.ErrorDetails; -import com.google.ads.googleads.v17.errors.GoogleAdsError; -import com.google.ads.googleads.v17.errors.GoogleAdsException; -import com.google.ads.googleads.v17.errors.GoogleAdsFailure; -import com.google.ads.googleads.v17.errors.InternalErrorEnum.InternalError; -import com.google.ads.googleads.v17.errors.QuotaErrorDetails; -import com.google.ads.googleads.v17.errors.QuotaErrorEnum.QuotaError; -import com.google.ads.googleads.v17.services.GoogleAdsRow; -import com.google.ads.googleads.v17.services.SearchGoogleAdsStreamRequest; -import com.google.ads.googleads.v17.services.SearchGoogleAdsStreamResponse; +import com.google.ads.googleads.v19.errors.AuthenticationErrorEnum.AuthenticationError; +import com.google.ads.googleads.v19.errors.ErrorCode; +import com.google.ads.googleads.v19.errors.ErrorDetails; +import com.google.ads.googleads.v19.errors.GoogleAdsError; +import com.google.ads.googleads.v19.errors.GoogleAdsException; +import com.google.ads.googleads.v19.errors.GoogleAdsFailure; +import com.google.ads.googleads.v19.errors.InternalErrorEnum.InternalError; +import com.google.ads.googleads.v19.errors.QuotaErrorDetails; +import com.google.ads.googleads.v19.errors.QuotaErrorEnum.QuotaError; +import com.google.ads.googleads.v19.services.GoogleAdsRow; +import com.google.ads.googleads.v19.services.SearchGoogleAdsStreamRequest; +import com.google.ads.googleads.v19.services.SearchGoogleAdsStreamResponse; import com.google.api.gax.grpc.GrpcStatusCode; import com.google.api.gax.rpc.ApiException; import com.google.protobuf.Duration; @@ -41,7 +41,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; -import org.apache.beam.sdk.io.googleads.GoogleAdsV17.RateLimitPolicyFactory; +import org.apache.beam.sdk.io.googleads.GoogleAdsIO.RateLimitPolicyFactory; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -61,8 +61,9 @@ import org.mockito.junit.MockitoJUnitRunner; @RunWith(Enclosed.class) -public class GoogleAdsV17Test { - static final RateLimitPolicyFactory TEST_POLICY_FACTORY = () -> new DummyRateLimitPolicy(); +public class GoogleAdsIOTest { + static final RateLimitPolicyFactory TEST_POLICY_FACTORY = + () -> new DummyRateLimitPolicy(); @RunWith(JUnit4.class) public static class ConstructionTests { @@ -73,7 +74,7 @@ public void testReadAllExpandWithDeveloperTokenFromBuilder() { pipeline .apply(Create.empty(new TypeDescriptor() {})) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .readAll() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withDeveloperToken("abc")); @@ -84,7 +85,7 @@ public void testReadAllExpandWithDeveloperTokenFromOptions() { pipeline.getOptions().as(GoogleAdsOptions.class).setGoogleAdsDeveloperToken("abc"); pipeline .apply(Create.empty(new TypeDescriptor() {})) - .apply(GoogleAdsIO.v17().readAll().withRateLimitPolicy(TEST_POLICY_FACTORY)); + .apply(GoogleAdsIO.current().readAll().withRateLimitPolicy(TEST_POLICY_FACTORY)); } @Test @@ -93,7 +94,7 @@ public void testReadAllExpandWithDeveloperTokenFromOptionsAndBuilder() { pipeline .apply(Create.empty(new TypeDescriptor() {})) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .readAll() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withDeveloperToken(null)); @@ -107,7 +108,7 @@ public void testReadAllExpandWithoutDeveloperToken() throws Exception { () -> pipeline .apply(Create.empty(new TypeDescriptor() {})) - .apply(GoogleAdsIO.v17().readAll().withRateLimitPolicy(TEST_POLICY_FACTORY))); + .apply(GoogleAdsIO.current().readAll().withRateLimitPolicy(TEST_POLICY_FACTORY))); } @Test @@ -118,7 +119,7 @@ public void testReadAllExpandWithoutRateLimitPolicy() throws Exception { () -> pipeline .apply(Create.empty(new TypeDescriptor() {})) - .apply(GoogleAdsIO.v17().readAll().withDeveloperToken("abc"))); + .apply(GoogleAdsIO.current().readAll().withDeveloperToken("abc"))); } @Test @@ -130,7 +131,7 @@ public void testReadAllExpandWithoutValidGoogleAdsClientFactory() throws Excepti pipeline .apply(Create.empty(new TypeDescriptor() {})) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .readAll() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withGoogleAdsClientFactory(null))); @@ -144,7 +145,7 @@ public void testReadAllExpandWithoutValidRateLimitPolicy() throws Exception { () -> pipeline .apply(Create.empty(new TypeDescriptor() {})) - .apply(GoogleAdsIO.v17().readAll().withRateLimitPolicy(null))); + .apply(GoogleAdsIO.current().readAll().withRateLimitPolicy(null))); } @Test @@ -152,7 +153,7 @@ public void testReadExpandWithDeveloperTokenFromBuilder() { pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withDeveloperToken("abc") @@ -161,7 +162,10 @@ public void testReadExpandWithDeveloperTokenFromBuilder() { pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17().read().withRateLimitPolicy(TEST_POLICY_FACTORY).withQuery("GAQL")); + GoogleAdsIO.current() + .read() + .withRateLimitPolicy(TEST_POLICY_FACTORY) + .withQuery("GAQL")); } @Test @@ -170,7 +174,10 @@ public void testReadExpandWithDeveloperTokenFromOptions() { pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17().read().withRateLimitPolicy(TEST_POLICY_FACTORY).withQuery("GAQL")); + GoogleAdsIO.current() + .read() + .withRateLimitPolicy(TEST_POLICY_FACTORY) + .withQuery("GAQL")); } @Test @@ -179,7 +186,7 @@ public void testReadExpandWithDeveloperTokenFromOptionsAndBuilder() { pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withDeveloperToken(null) @@ -195,7 +202,7 @@ public void testReadExpandWithoutDeveloperToken() throws Exception { pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withQuery("GAQL"))); @@ -209,7 +216,7 @@ public void testReadExpandWithoutQuery() throws Exception { () -> pipeline .apply(Create.empty(TypeDescriptors.strings())) - .apply(GoogleAdsIO.v17().read().withRateLimitPolicy(TEST_POLICY_FACTORY))); + .apply(GoogleAdsIO.current().read().withRateLimitPolicy(TEST_POLICY_FACTORY))); } @Test @@ -220,7 +227,7 @@ public void testReadExpandWithoutRateLimitPolicy() throws Exception { () -> pipeline .apply(Create.empty(TypeDescriptors.strings())) - .apply(GoogleAdsIO.v17().read().withDeveloperToken("abc").withQuery("GAQL"))); + .apply(GoogleAdsIO.current().read().withDeveloperToken("abc").withQuery("GAQL"))); } @Test @@ -232,7 +239,7 @@ public void testReadExpandWithoutValidGoogleAdsClientFactory() throws Exception pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withQuery("GAQL") @@ -248,7 +255,7 @@ public void testReadExpandWithoutValidQuery() throws Exception { pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withQuery(null))); @@ -260,7 +267,7 @@ public void testReadExpandWithoutValidQuery() throws Exception { pipeline .apply(Create.empty(TypeDescriptors.strings())) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withRateLimitPolicy(TEST_POLICY_FACTORY) .withQuery(""))); @@ -274,7 +281,7 @@ public void testReadExpandWithoutValidRateLimitPolicy() throws Exception { () -> pipeline .apply(Create.empty(TypeDescriptors.strings())) - .apply(GoogleAdsIO.v17().read().withQuery("GAQL").withRateLimitPolicy(null))); + .apply(GoogleAdsIO.current().read().withQuery("GAQL").withRateLimitPolicy(null))); } } @@ -286,15 +293,15 @@ public static class ExecutionTests { public void init() { GoogleAdsOptions options = pipeline.getOptions().as(GoogleAdsOptions.class); options.setGoogleAdsCredentialFactoryClass(NoopCredentialFactory.class); - synchronized (GoogleAdsV17.ReadAll.ReadAllFn.class) { - GoogleAdsV17.ReadAll.ReadAllFn.sleeper = (long millis) -> {}; + synchronized (GoogleAdsV19.ReadAll.ReadAllFn.class) { + GoogleAdsV19.ReadAll.ReadAllFn.sleeper = (long millis) -> {}; } } @Test @Category(NeedsRunner.class) public void testRead() { - when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V17 + when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V19 .searchStreamCallable() .call(any(SearchGoogleAdsStreamRequest.class)) .iterator()) @@ -309,7 +316,7 @@ public void testRead() { pipeline .apply(Create.of("123")) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withGoogleAdsClientFactory(new MockGoogleAdsClientFactory()) .withRateLimitPolicy(TEST_POLICY_FACTORY) @@ -323,7 +330,7 @@ public void testRead() { @Test @Category(NeedsRunner.class) public void testReadWithFailureFromMaxRetriesExceeded() throws Exception { - when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V17 + when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V19 .searchStreamCallable() .call(any(SearchGoogleAdsStreamRequest.class))) .thenThrow( @@ -341,7 +348,7 @@ public void testReadWithFailureFromMaxRetriesExceeded() throws Exception { pipeline .apply(Create.of("123")) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withGoogleAdsClientFactory(new MockGoogleAdsClientFactory()) .withRateLimitPolicy(TEST_POLICY_FACTORY) @@ -362,7 +369,7 @@ public void testReadWithFailureFromMaxRetriesExceeded() throws Exception { @Test @Category(NeedsRunner.class) public void testReadWithFailureFromNonRetryableError() throws Exception { - when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V17 + when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V19 .searchStreamCallable() .call(any(SearchGoogleAdsStreamRequest.class))) .thenThrow( @@ -381,7 +388,7 @@ public void testReadWithFailureFromNonRetryableError() throws Exception { pipeline .apply(Create.of("123")) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withGoogleAdsClientFactory(new MockGoogleAdsClientFactory()) .withRateLimitPolicy(TEST_POLICY_FACTORY) @@ -395,7 +402,7 @@ public void testReadWithFailureFromNonRetryableError() throws Exception { pipeline::run); Assert.assertEquals(IOException.class, exception.getCause().getClass()); Assert.assertEquals( - "com.google.ads.googleads.v17.errors.GoogleAdsException: errors {\n" + "com.google.ads.googleads.v19.errors.GoogleAdsException: errors {\n" + " error_code {\n" + " authentication_error: OAUTH_TOKEN_REVOKED\n" + " }\n" @@ -406,7 +413,7 @@ public void testReadWithFailureFromNonRetryableError() throws Exception { @Test @Category(NeedsRunner.class) public void testReadWithRecoveryFromInternalError() throws Exception { - when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V17 + when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V19 .searchStreamCallable() .call(any(SearchGoogleAdsStreamRequest.class)) .iterator()) @@ -443,7 +450,7 @@ public void testReadWithRecoveryFromInternalError() throws Exception { pipeline .apply(Create.of("123")) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withGoogleAdsClientFactory(new MockGoogleAdsClientFactory()) .withRateLimitPolicy(TEST_POLICY_FACTORY) @@ -457,7 +464,7 @@ public void testReadWithRecoveryFromInternalError() throws Exception { @Test @Category(NeedsRunner.class) public void testReadWithRecoveryFromQuotaErrorWithRetryDelay() throws Exception { - when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V17 + when(MockGoogleAdsClientFactory.GOOGLE_ADS_SERVICE_STUB_V19 .searchStreamCallable() .call(any(SearchGoogleAdsStreamRequest.class)) .iterator()) @@ -505,7 +512,7 @@ public void testReadWithRecoveryFromQuotaErrorWithRetryDelay() throws Exception pipeline .apply(Create.of("123")) .apply( - GoogleAdsIO.v17() + GoogleAdsIO.current() .read() .withGoogleAdsClientFactory(new MockGoogleAdsClientFactory()) .withRateLimitPolicy(TEST_POLICY_FACTORY) diff --git a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/MockGoogleAdsClientFactory.java b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/MockGoogleAdsClientFactory.java index 4ed1f87788f5..6ff6ef92e9e0 100644 --- a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/MockGoogleAdsClientFactory.java +++ b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/MockGoogleAdsClientFactory.java @@ -23,12 +23,12 @@ import static org.mockito.Mockito.withSettings; import com.google.ads.googleads.lib.GoogleAdsClient; -import com.google.ads.googleads.v17.services.GoogleAdsServiceClient; -import com.google.ads.googleads.v17.services.stub.GoogleAdsServiceStub; +import com.google.ads.googleads.v19.services.GoogleAdsServiceClient; +import com.google.ads.googleads.v19.services.stub.GoogleAdsServiceStub; import org.checkerframework.checker.nullness.qual.Nullable; class MockGoogleAdsClientFactory implements GoogleAdsClientFactory { - static final GoogleAdsServiceStub GOOGLE_ADS_SERVICE_STUB_V17 = + static final GoogleAdsServiceStub GOOGLE_ADS_SERVICE_STUB_V19 = mock(GoogleAdsServiceStub.class, withSettings().defaultAnswer(RETURNS_DEEP_STUBS)); @Override @@ -39,8 +39,8 @@ public GoogleAdsClient newGoogleAdsClient( @Nullable Long loginCustomerId) { GoogleAdsClient mockGoogleAdsClient = mock(GoogleAdsClient.class, withSettings().defaultAnswer(RETURNS_DEEP_STUBS)); - when(mockGoogleAdsClient.getVersion17().createGoogleAdsServiceClient()) - .thenReturn(GoogleAdsServiceClient.create(GOOGLE_ADS_SERVICE_STUB_V17)); + when(mockGoogleAdsClient.getVersion19().createGoogleAdsServiceClient()) + .thenReturn(GoogleAdsServiceClient.create(GOOGLE_ADS_SERVICE_STUB_V19)); return mockGoogleAdsClient; } }