From 47e4d21f56dafead89044aa59ada43d9eb9be067 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 3 Feb 2022 11:09:16 -0800 Subject: [PATCH 01/52] Adding gRPC compression support to the library --- .../com/google/cloud/pubsub/v1/Publisher.java | 44 ++++++++++++++++--- 1 file changed, 37 insertions(+), 7 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 1ed5e2254..dcb683c21 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -35,6 +35,7 @@ import com.google.api.gax.core.ExecutorProvider; import com.google.api.gax.core.FixedExecutorProvider; import com.google.api.gax.core.InstantiatingExecutorProvider; +import com.google.api.gax.grpc.GrpcCallContext; import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.HeaderProvider; import com.google.api.gax.rpc.NoHeaderProvider; @@ -67,6 +68,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.logging.Level; import java.util.logging.Logger; +import io.grpc.CallOptions; import org.threeten.bp.Duration; /** @@ -114,6 +116,11 @@ public class Publisher implements PublisherInterface { private MessageFlowController flowController = null; + private final boolean enableCompression; + + /** The message is compressed when its size (in bytes) is above the threshold. */ + private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte (https://en.wikipedia.org/wiki/Kilobyte) + /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { return 1000L; @@ -191,6 +198,8 @@ private Publisher(Builder builder) throws IOException { backgroundResources = new BackgroundResourceAggregation(backgroundResourceList); shutdown = new AtomicBoolean(false); messagesWaiter = new Waiter(); + + enableCompression = builder.enableCompression; } /** Topic which the publisher publishes to. */ @@ -432,13 +441,25 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { - return publisherStub - .publishCallable() - .futureCall( - PublishRequest.newBuilder() - .setTopic(topicName) - .addAllMessages(outstandingBatch.getMessages()) - .build()); + if (enableCompression && outstandingBatch.batchSizeBytes >= MSG_COMPRESSION_THRESHOLD_BYTES) { + GrpcCallContext context = GrpcCallContext.createDefault(); + context = context.withCallOptions(CallOptions.DEFAULT.withCompression("gzip")); + return publisherStub + .publishCallable() + .futureCall( + PublishRequest.newBuilder() + .setTopic(topicName) + .addAllMessages(outstandingBatch.getMessages()) + .build(), context); + } else { + return publisherStub + .publishCallable() + .futureCall( + PublishRequest.newBuilder() + .setTopic(topicName) + .addAllMessages(outstandingBatch.getMessages()) + .build()); + } } private void publishOutstandingBatch(final OutstandingBatch outstandingBatch) { @@ -689,6 +710,7 @@ public static final class Builder { InstantiatingExecutorProvider.newBuilder() .setExecutorThreadCount(THREADS_PER_CPU * Runtime.getRuntime().availableProcessors()) .build(); + static final boolean DEFAULT_ENABLE_COMPRESSION = false; String topicName; private String endpoint = PublisherStubSettings.getDefaultEndpoint(); @@ -718,6 +740,8 @@ public PubsubMessage apply(PubsubMessage input) { } }; + private boolean enableCompression = DEFAULT_ENABLE_COMPRESSION; + private Builder(String topic) { this.topicName = Preconditions.checkNotNull(topic); } @@ -829,6 +853,12 @@ public Builder setEndpoint(String endpoint) { return this; } + /** Gives the ability to enable gRPC compression. */ + public Builder setEnableCompression(boolean enableCompression) { + this.enableCompression = enableCompression; + return this; + } + /** Returns the default BatchingSettings used by the client if settings are not provided. */ public static BatchingSettings getDefaultBatchingSettings() { return DEFAULT_BATCHING_SETTINGS; From 7190fcaae749f449c0e37efef08c4f9759e9079d Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 3 Feb 2022 11:11:23 -0800 Subject: [PATCH 02/52] Minor comment fix --- .../src/main/java/com/google/cloud/pubsub/v1/Publisher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index dcb683c21..7bbfc9414 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -119,7 +119,7 @@ public class Publisher implements PublisherInterface { private final boolean enableCompression; /** The message is compressed when its size (in bytes) is above the threshold. */ - private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte (https://en.wikipedia.org/wiki/Kilobyte) + private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { From 8d05ba21fe3eb4e3fc98203e0237118063ecf2f8 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 3 Feb 2022 22:55:30 +0000 Subject: [PATCH 03/52] Formatting the code --- .../main/java/com/google/cloud/pubsub/v1/Publisher.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 7bbfc9414..8b828ee7d 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -51,6 +51,7 @@ import com.google.pubsub.v1.PubsubMessage; import com.google.pubsub.v1.TopicName; import com.google.pubsub.v1.TopicNames; +import io.grpc.CallOptions; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -68,7 +69,6 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.logging.Level; import java.util.logging.Logger; -import io.grpc.CallOptions; import org.threeten.bp.Duration; /** @@ -450,9 +450,10 @@ private ApiFuture publishCall(OutstandingBatch outstandingBatch PublishRequest.newBuilder() .setTopic(topicName) .addAllMessages(outstandingBatch.getMessages()) - .build(), context); - } else { - return publisherStub + .build(), + context); + } else { + return publisherStub .publishCallable() .futureCall( PublishRequest.newBuilder() From 13a071df8dd0f0739534474dd778e3ba9634416c Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 7 Feb 2022 18:40:16 -0500 Subject: [PATCH 04/52] Adding unit test for compression --- .../com/google/cloud/pubsub/v1/Publisher.java | 8 ++++-- .../cloud/pubsub/v1/PublisherImplTest.java | 25 +++++++++++++++++++ 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 8b828ee7d..df480b907 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -147,6 +147,7 @@ private Publisher(Builder builder) throws IOException { this.enableMessageOrdering = builder.enableMessageOrdering; this.messageTransform = builder.messageTransform; + this.enableCompression = builder.enableCompression; messagesBatches = new HashMap<>(); messagesBatchLock = new ReentrantLock(); @@ -198,8 +199,6 @@ private Publisher(Builder builder) throws IOException { backgroundResources = new BackgroundResourceAggregation(backgroundResourceList); shutdown = new AtomicBoolean(false); messagesWaiter = new Waiter(); - - enableCompression = builder.enableCompression; } /** Topic which the publisher publishes to. */ @@ -212,6 +211,11 @@ public String getTopicNameString() { return topicName; } + /** Returns True/False if compression is enabled or disabled respectively. */ + public boolean getEnableCompression() { + return this.enableCompression; + } + /** * Schedules the publishing of a message. The publishing of the message may occur immediately or * be delayed based on the publisher batching options. diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java index c41931de6..9a8ee9494 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java @@ -282,6 +282,31 @@ public void testPublishMixedSizeAndDuration() throws Exception { shutdownTestPublisher(publisher); } + @Test + public void testPublishWithCompression() throws Exception { + Publisher publisher = + getTestPublisherBuilder() + .setBatchingSettings( + Publisher.Builder.DEFAULT_BATCHING_SETTINGS + .toBuilder() + .setElementCountThreshold(2L) + .setDelayThreshold(Duration.ofSeconds(100)) + .build()) + .setEnableCompression(true) + .build(); + assertTrue(publisher.getEnableCompression()); + + testPublisherServiceImpl + .addPublishResponse(PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); + ApiFuture publishFuture1 = sendTestMessage(publisher, "A"); + ApiFuture publishFuture2 = sendTestMessage(publisher, "B"); + assertEquals("1", publishFuture1.get()); + assertEquals("2", publishFuture2.get()); + + fakeExecutor.advanceTime(Duration.ofSeconds(100)); + shutdownTestPublisher(publisher); + } + private ApiFuture sendTestMessage(Publisher publisher, String data) { return publisher.publish( PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(data)).build()); From d7eb3dce8204e53108542e86b92958b1014d6a15 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 7 Feb 2022 21:01:33 -0500 Subject: [PATCH 05/52] Adding integration test for compression --- .../google/cloud/pubsub/it/ITPubSubTest.java | 74 +++++++++++++++++++ .../cloud/pubsub/v1/PublisherImplTest.java | 4 +- 2 files changed, 76 insertions(+), 2 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index c72d52d3d..d25156bf4 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -403,6 +403,70 @@ public void failed(Subscriber.State from, Throwable failure) { topicAdminClient.deleteTopic(topicName); } + @Test + public void testPublishSubscribeWithCompression() throws Exception { + TopicName topicName = + TopicName.newBuilder() + .setProject(projectId) + .setTopic(formatForTest("testing-compression-topic")) + .build(); + SubscriptionName subscriptionName = + SubscriptionName.of(projectId, formatForTest("testing-compression-subscription")); + + topicAdminClient.createTopic(topicName); + + subscriptionAdminClient.createSubscription( + getSubscription(subscriptionName, topicName, PushConfig.newBuilder().build(), 10)); + + final BlockingQueue receiveQueue = new LinkedBlockingQueue<>(); + Subscriber subscriber = + Subscriber.newBuilder( + subscriptionName.toString(), + new MessageReceiver() { + @Override + public void receiveMessage( + final PubsubMessage message, final AckReplyConsumer consumer) { + receiveQueue.offer(MessageAndConsumer.create(message, consumer)); + } + }) + .build(); + subscriber.addListener( + new Subscriber.Listener() { + public void failed(Subscriber.State from, Throwable failure) { + receiveQueue.offer(failure); + } + }, + MoreExecutors.directExecutor()); + subscriber.startAsync(); + + Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + + String msg1 = generateMessage("msg1", 1000); + String msg2 = generateMessage("msg2", 1500); + publisher + .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(msg1)).build()) + .get(); + publisher + .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(msg2)).build()) + .get(); + publisher.shutdown(); + publisher.awaitTermination(1, TimeUnit.MINUTES); + + // Ack the first message. + MessageAndConsumer toAck1 = pollQueue(receiveQueue); + assertThat(toAck1.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg1)); + toAck1.consumer().ack(); + + // Ack the second message. + MessageAndConsumer toAck2 = pollQueue(receiveQueue); + assertThat(toAck2.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg2)); + toAck2.consumer().ack(); + + subscriber.stopAsync().awaitTerminated(); + subscriptionAdminClient.deleteSubscription(subscriptionName); + topicAdminClient.deleteTopic(topicName); + } + private MessageAndConsumer pollQueueMessageAndConsumer(BlockingQueue queue) throws InterruptedException { Object obj = pollQueue(queue); @@ -434,4 +498,14 @@ private Object pollQueue(BlockingQueue queue) throws InterruptedExceptio return obj; } + + /** Generates message of given bytes by repeatedly concatenating a token. */ + private String generateMessage(String token, int bytes) { + String result = ""; + int tokenBytes = token.length(); + for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { + result = result.concat(token); + } + return result; + } } diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java index 9a8ee9494..1374ffad4 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java @@ -296,8 +296,8 @@ public void testPublishWithCompression() throws Exception { .build(); assertTrue(publisher.getEnableCompression()); - testPublisherServiceImpl - .addPublishResponse(PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); + testPublisherServiceImpl.addPublishResponse( + PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); ApiFuture publishFuture1 = sendTestMessage(publisher, "A"); ApiFuture publishFuture2 = sendTestMessage(publisher, "B"); assertEquals("1", publishFuture1.get()); From 49896ac3fd4f29cf94c6c34ad7ca24e35c8ee504 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Tue, 8 Feb 2022 12:09:37 -0500 Subject: [PATCH 06/52] Formatting --- .../java/com/google/cloud/pubsub/it/ITPubSubTest.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index d25156bf4..afc94d42b 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -40,11 +40,7 @@ import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; +import org.junit.*; import org.junit.rules.Timeout; public class ITPubSubTest { @@ -439,7 +435,10 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + Publisher publisher = + Publisher.newBuilder(topicName) + .setEnableCompression(true) + .build(); String msg1 = generateMessage("msg1", 1000); String msg2 = generateMessage("msg2", 1500); From f9bea54bd676e78472c54148af4921deec5bccfd Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Tue, 8 Feb 2022 15:02:47 -0500 Subject: [PATCH 07/52] Refactoring integration tests to add support for overriding endpoint --- .../google/cloud/pubsub/it/ITPubSubTest.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index afc94d42b..cf2026d41 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -24,7 +24,14 @@ import com.google.api.gax.rpc.PermissionDeniedException; import com.google.auto.value.AutoValue; import com.google.cloud.ServiceOptions; -import com.google.cloud.pubsub.v1.*; +import com.google.cloud.pubsub.v1.AckReplyConsumer; +import com.google.cloud.pubsub.v1.MessageReceiver; +import com.google.cloud.pubsub.v1.Publisher; +import com.google.cloud.pubsub.v1.Subscriber; +import com.google.cloud.pubsub.v1.SubscriptionAdminClient; +import com.google.cloud.pubsub.v1.SubscriptionAdminSettings; +import com.google.cloud.pubsub.v1.TopicAdminClient; +import com.google.cloud.pubsub.v1.TopicAdminSettings; import com.google.common.util.concurrent.MoreExecutors; import com.google.iam.v1.Binding; import com.google.iam.v1.GetIamPolicyRequest; @@ -49,6 +56,7 @@ public class ITPubSubTest { private static TopicAdminClient topicAdminClient; private static SubscriptionAdminClient subscriptionAdminClient; private static String projectId; + private static String endPoint = "us-east1-pubsub.googleapis.com:443"; private static final boolean IS_VPC_TEST = System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC") != null && System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC").equalsIgnoreCase("true"); @@ -83,8 +91,11 @@ static MessageAndConsumerWithResponse create( @BeforeClass public static void setupClass() throws Exception { - topicAdminClient = TopicAdminClient.create(); - subscriptionAdminClient = SubscriptionAdminClient.create(); + topicAdminClient = + TopicAdminClient.create(TopicAdminSettings.newBuilder().setEndpoint(endPoint).build()); + subscriptionAdminClient = + SubscriptionAdminClient.create( + SubscriptionAdminSettings.newBuilder().setEndpoint(endPoint).build()); projectId = ServiceOptions.getDefaultProjectId(); } @@ -205,6 +216,7 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) + .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -215,7 +227,7 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = Publisher.newBuilder(topicName).build(); + Publisher publisher = Publisher.newBuilder(topicName).setEndpoint(endPoint).build(); publisher .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8("msg1")).build()) .get(); @@ -425,6 +437,7 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) + .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -436,9 +449,7 @@ public void failed(Subscriber.State from, Throwable failure) { subscriber.startAsync(); Publisher publisher = - Publisher.newBuilder(topicName) - .setEnableCompression(true) - .build(); + Publisher.newBuilder(topicName).setEndpoint(endPoint).setEnableCompression(true).build(); String msg1 = generateMessage("msg1", 1000); String msg2 = generateMessage("msg2", 1500); From be4f7b90163bc9e414c335911d611ec6f1d26cfa Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 9 Feb 2022 17:01:03 -0500 Subject: [PATCH 08/52] Adding sample for publish with compression; Updating README --- README.md | 1 + .../pubsub/PublishWithCompressionExample.java | 76 +++++++++++++++++++ 2 files changed, 77 insertions(+) create mode 100644 samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java diff --git a/README.md b/README.md index 382a5fe8f..13bad9a07 100644 --- a/README.md +++ b/README.md @@ -272,6 +272,7 @@ Samples are in the [`samples/`](https://github.com/googleapis/java-pubsub/tree/m | Publish With Flow Control Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | | Publish With Ordering Keys | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | | Publish With Retry Settings Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | +| Publish With Compression Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | | Publisher Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublisherExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublisherExample.java) | | Receive Messages With Delivery Attempts Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | | Remove Dead Letter Policy Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java new file mode 100644 index 000000000..da2e5c318 --- /dev/null +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -0,0 +1,76 @@ +/* + * Copyright 2020 Google LLC + * + * Licensed 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 pubsub; + +// [START pubsub_publish_with_compression] +import com.google.api.core.ApiFuture; +import com.google.cloud.pubsub.v1.Publisher; +import com.google.protobuf.ByteString; +import com.google.pubsub.v1.PubsubMessage; +import com.google.pubsub.v1.TopicName; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +public class PublishWithCompressionExample { + public static void main(String... args) throws Exception { + // TODO(developer): Replace these variables before running the sample. + String projectId = "your-project-id"; + // Choose an existing topic. + String topicId = "your-topic-id"; + + publishWithCompression(projectId, topicId); + } + + public static void publishWithCompression(String projectId, String topicId) + throws IOException, ExecutionException, InterruptedException { + TopicName topicName = TopicName.of(projectId, topicId); + + Publisher publisher = null; + try { + // Create a publisher instance with default settings bound to the topic + publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + + // Compression works only for messages of size >= 1KB + String message = generateMessage("Hello!", 2000); + ByteString data = ByteString.copyFromUtf8(message); + PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); + + // Once published, returns a server-assigned message id (unique within the topic) + ApiFuture messageIdFuture = publisher.publish(pubsubMessage); + String messageId = messageIdFuture.get(); + System.out.println("Published message ID: " + messageId); + } finally { + if (publisher != null) { + // When finished with the publisher, shutdown to free up resources. + publisher.shutdown(); + publisher.awaitTermination(1, TimeUnit.MINUTES); + } + } + } + + /** Generate message of given bytes by repeatedly concatenating a token.* */ + private static String generateMessage(String token, int bytes) { + String result = ""; + int tokenBytes = token.length(); + for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { + result = result.concat(token); + } + return result; + } +} +// [END pubsub_publish_with_compression] From 525a738cfc25e6f49afb4ed27c4a32bee4ab83e7 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 9 Feb 2022 17:48:57 -0500 Subject: [PATCH 09/52] Adding integration test for compression sample --- .../src/main/java/pubsub/PublishWithCompressionExample.java | 6 +++--- samples/snippets/src/test/java/pubsub/PublisherIT.java | 5 +++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index da2e5c318..40273e2f8 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -33,10 +33,10 @@ public static void main(String... args) throws Exception { // Choose an existing topic. String topicId = "your-topic-id"; - publishWithCompression(projectId, topicId); + publishWithCompressionExample(projectId, topicId); } - public static void publishWithCompression(String projectId, String topicId) + public static void publishWithCompressionExample(String projectId, String topicId) throws IOException, ExecutionException, InterruptedException { TopicName topicName = TopicName.of(projectId, topicId); @@ -53,7 +53,7 @@ public static void publishWithCompression(String projectId, String topicId) // Once published, returns a server-assigned message id (unique within the topic) ApiFuture messageIdFuture = publisher.publish(pubsubMessage); String messageId = messageIdFuture.get(); - System.out.println("Published message ID: " + messageId); + System.out.println("Published compressed message ID: " + messageId); } finally { if (publisher != null) { // When finished with the publisher, shutdown to free up resources. diff --git a/samples/snippets/src/test/java/pubsub/PublisherIT.java b/samples/snippets/src/test/java/pubsub/PublisherIT.java index b22912d47..48525e7d6 100644 --- a/samples/snippets/src/test/java/pubsub/PublisherIT.java +++ b/samples/snippets/src/test/java/pubsub/PublisherIT.java @@ -124,5 +124,10 @@ public void testPublisher() throws Exception { for (int i = 1; i <= 4; i++) { assertThat(bout.toString()).contains("message" + i); } + + bout.reset(); + // Test publish with compression. + PublishWithCompressionExample.publishWithCompressionExample(projectId, topicId); + assertThat(bout.toString()).contains("Published compressed message ID: "); } } From b6d51d05c00c172e72eb944a2658ab4e459d80b8 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 16 Feb 2022 11:55:35 -0500 Subject: [PATCH 10/52] Adding parameter compressionBytesThreshold to Publisher; Adding logging support in the compression example --- .../com/google/cloud/pubsub/v1/Publisher.java | 15 ++++-- .../pubsub/PublishWithCompressionExample.java | 50 +++++++++++++++---- 2 files changed, 51 insertions(+), 14 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index df480b907..fbd4ceb6f 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -117,9 +117,7 @@ public class Publisher implements PublisherInterface { private MessageFlowController flowController = null; private final boolean enableCompression; - - /** The message is compressed when its size (in bytes) is above the threshold. */ - private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte + private final long compressionBytesThreshold; /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { @@ -148,6 +146,7 @@ private Publisher(Builder builder) throws IOException { this.enableMessageOrdering = builder.enableMessageOrdering; this.messageTransform = builder.messageTransform; this.enableCompression = builder.enableCompression; + this.compressionBytesThreshold = builder.compressionBytesThreshold; messagesBatches = new HashMap<>(); messagesBatchLock = new ReentrantLock(); @@ -445,7 +444,7 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { - if (enableCompression && outstandingBatch.batchSizeBytes >= MSG_COMPRESSION_THRESHOLD_BYTES) { + if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { GrpcCallContext context = GrpcCallContext.createDefault(); context = context.withCallOptions(CallOptions.DEFAULT.withCompression("gzip")); return publisherStub @@ -682,6 +681,7 @@ public static final class Builder { // Meaningful defaults. static final long DEFAULT_ELEMENT_COUNT_THRESHOLD = 100L; static final long DEFAULT_REQUEST_BYTES_THRESHOLD = 1000L; // 1 kB + static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; static final Duration DEFAULT_DELAY_THRESHOLD = Duration.ofMillis(1); private static final Duration DEFAULT_INITIAL_RPC_TIMEOUT = Duration.ofSeconds(5); private static final Duration DEFAULT_MAX_RPC_TIMEOUT = Duration.ofSeconds(60); @@ -746,6 +746,7 @@ public PubsubMessage apply(PubsubMessage input) { }; private boolean enableCompression = DEFAULT_ENABLE_COMPRESSION; + private long compressionBytesThreshold = DEFAULT_COMPRESSION_BYTES_THRESHOLD; private Builder(String topic) { this.topicName = Preconditions.checkNotNull(topic); @@ -864,6 +865,12 @@ public Builder setEnableCompression(boolean enableCompression) { return this; } + /** Gives the ability to set the threshold in bytes above which gRPC compression happens. */ + public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { + this.compressionBytesThreshold = compressionBytesThreshold; + return this; + } + /** Returns the default BatchingSettings used by the client if settings are not provided. */ public static BatchingSettings getDefaultBatchingSettings() { return DEFAULT_BATCHING_SETTINGS; diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 40273e2f8..36eb03933 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -22,17 +22,24 @@ import com.google.protobuf.ByteString; import com.google.pubsub.v1.PubsubMessage; import com.google.pubsub.v1.TopicName; +import java.io.ByteArrayInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.logging.LogManager; public class PublishWithCompressionExample { + public static void main(String... args) throws Exception { // TODO(developer): Replace these variables before running the sample. String projectId = "your-project-id"; - // Choose an existing topic. String topicId = "your-topic-id"; + boolean allowLogging = false; // Set to true to get the stdout logs + if (allowLogging) { + setUpLogs(); + } publishWithCompressionExample(projectId, topicId); } @@ -42,12 +49,11 @@ public static void publishWithCompressionExample(String projectId, String topicI Publisher publisher = null; try { - // Create a publisher instance with default settings bound to the topic + // Create a publisher instance bound to the topic with compression enabled and other default + // settings publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); - // Compression works only for messages of size >= 1KB - String message = generateMessage("Hello!", 2000); - ByteString data = ByteString.copyFromUtf8(message); + ByteString data = generateData("Hello!", 2000); PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); // Once published, returns a server-assigned message id (unique within the topic) @@ -63,14 +69,38 @@ public static void publishWithCompressionExample(String projectId, String topicI } } - /** Generate message of given bytes by repeatedly concatenating a token.* */ - private static String generateMessage(String token, int bytes) { - String result = ""; + // Generates data of given bytes by repeatedly concatenating a token. + // TODO(developer): Replace this method with your own data generation logic + private static ByteString generateData(String token, int bytes) { + String message = ""; int tokenBytes = token.length(); for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { - result = result.concat(token); + message = message.concat(token); } - return result; + return ByteString.copyFromUtf8(message); + } + + private static void setUpLogs() throws IOException { + String handlers = "handlers = java.util.logging.ConsoleHandler"; + String handlerLevelProp = "java.util.logging.ConsoleHandler.level = ALL"; + String fineProp = ".level = FINE"; + String handlerFormatterProp = + "java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter"; + String format = "java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n"; + + LogManager.getLogManager() + .readConfiguration( + new ByteArrayInputStream( + (handlers + + "\n" + + handlerLevelProp + + "\n" + + fineProp + + "\n" + + handlerFormatterProp + + "\n" + + format) + .getBytes(StandardCharsets.UTF_8))); } } // [END pubsub_publish_with_compression] From e0819e536032fc96495b6e248740db00eea5ff4a Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 2 Mar 2022 12:28:41 -0500 Subject: [PATCH 11/52] Addressing PR comments --- .../com/google/cloud/pubsub/v1/Publisher.java | 8 ++++++-- .../google/cloud/pubsub/it/ITPubSubTest.java | 14 ++++---------- .../pubsub/PublishWithCompressionExample.java | 19 ++++++++++++++----- 3 files changed, 24 insertions(+), 17 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index fbd4ceb6f..454729f71 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -681,7 +681,6 @@ public static final class Builder { // Meaningful defaults. static final long DEFAULT_ELEMENT_COUNT_THRESHOLD = 100L; static final long DEFAULT_REQUEST_BYTES_THRESHOLD = 1000L; // 1 kB - static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; static final Duration DEFAULT_DELAY_THRESHOLD = Duration.ofMillis(1); private static final Duration DEFAULT_INITIAL_RPC_TIMEOUT = Duration.ofSeconds(5); private static final Duration DEFAULT_MAX_RPC_TIMEOUT = Duration.ofSeconds(60); @@ -716,6 +715,7 @@ public static final class Builder { .setExecutorThreadCount(THREADS_PER_CPU * Runtime.getRuntime().availableProcessors()) .build(); static final boolean DEFAULT_ENABLE_COMPRESSION = false; + static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; String topicName; private String endpoint = PublisherStubSettings.getDefaultEndpoint(); @@ -865,7 +865,11 @@ public Builder setEnableCompression(boolean enableCompression) { return this; } - /** Gives the ability to set the threshold in bytes above which gRPC compression happens. */ + /** + * Gives the ability to set the threshold in bytes above which gRPC compression happens. But to + * enable gRPC compression at the first place, setEnableCompression(true) should be called along + * with this method. + */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { this.compressionBytesThreshold = compressionBytesThreshold; return this; diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index cf2026d41..c5e512e65 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -56,7 +56,6 @@ public class ITPubSubTest { private static TopicAdminClient topicAdminClient; private static SubscriptionAdminClient subscriptionAdminClient; private static String projectId; - private static String endPoint = "us-east1-pubsub.googleapis.com:443"; private static final boolean IS_VPC_TEST = System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC") != null && System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC").equalsIgnoreCase("true"); @@ -91,11 +90,9 @@ static MessageAndConsumerWithResponse create( @BeforeClass public static void setupClass() throws Exception { - topicAdminClient = - TopicAdminClient.create(TopicAdminSettings.newBuilder().setEndpoint(endPoint).build()); + topicAdminClient = TopicAdminClient.create(TopicAdminSettings.newBuilder().build()); subscriptionAdminClient = - SubscriptionAdminClient.create( - SubscriptionAdminSettings.newBuilder().setEndpoint(endPoint).build()); + SubscriptionAdminClient.create(SubscriptionAdminSettings.newBuilder().build()); projectId = ServiceOptions.getDefaultProjectId(); } @@ -216,7 +213,6 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) - .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -227,7 +223,7 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = Publisher.newBuilder(topicName).setEndpoint(endPoint).build(); + Publisher publisher = Publisher.newBuilder(topicName).build(); publisher .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8("msg1")).build()) .get(); @@ -437,7 +433,6 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) - .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -448,8 +443,7 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = - Publisher.newBuilder(topicName).setEndpoint(endPoint).setEnableCompression(true).build(); + Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); String msg1 = generateMessage("msg1", 1000); String msg2 = generateMessage("msg2", 1500); diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 36eb03933..14becb78f 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -49,9 +49,13 @@ public static void publishWithCompressionExample(String projectId, String topicI Publisher publisher = null; try { - // Create a publisher instance bound to the topic with compression enabled and other default - // settings - publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + // Create a publisher instance bound to the topic with compression enabled and a compression + // bytes threshold. + publisher = + Publisher.newBuilder(topicName) + .setEnableCompression(true) + .setCompressionBytesThreshold(500) + .build(); ByteString data = generateData("Hello!", 2000); PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); @@ -59,7 +63,7 @@ public static void publishWithCompressionExample(String projectId, String topicI // Once published, returns a server-assigned message id (unique within the topic) ApiFuture messageIdFuture = publisher.publish(pubsubMessage); String messageId = messageIdFuture.get(); - System.out.println("Published compressed message ID: " + messageId); + System.out.println("Published compressed message, ID:" + messageId); } finally { if (publisher != null) { // When finished with the publisher, shutdown to free up resources. @@ -69,7 +73,7 @@ public static void publishWithCompressionExample(String projectId, String topicI } } - // Generates data of given bytes by repeatedly concatenating a token. + /** Generates data of given bytes by repeatedly concatenating a token. */ // TODO(developer): Replace this method with your own data generation logic private static ByteString generateData(String token, int bytes) { String message = ""; @@ -80,6 +84,11 @@ private static ByteString generateData(String token, int bytes) { return ByteString.copyFromUtf8(message); } + /** + * Sets up logging to observe the outbound data (and its length) over the network to analyze the + * effectiveness of compression. A sample log line: + * [2022-03-02] FINE [id:..] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01... + */ private static void setUpLogs() throws IOException { String handlers = "handlers = java.util.logging.ConsoleHandler"; String handlerLevelProp = "java.util.logging.ConsoleHandler.level = ALL"; From 6412cf2e8b2cd2fcd8d5367cde5797ebd77065b1 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 2 Mar 2022 12:31:46 -0500 Subject: [PATCH 12/52] Addressing checkstyle --- .../src/main/java/pubsub/PublishWithCompressionExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 14becb78f..e625e335d 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -87,7 +87,7 @@ private static ByteString generateData(String token, int bytes) { /** * Sets up logging to observe the outbound data (and its length) over the network to analyze the * effectiveness of compression. A sample log line: - * [2022-03-02] FINE [id:..] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01... + * [2022-03-02] FINE [] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01.. */ private static void setUpLogs() throws IOException { String handlers = "handlers = java.util.logging.ConsoleHandler"; From 014fe2ae7888b228cd4304b58e594a8e266b6d8d Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 2 Mar 2022 14:56:50 -0500 Subject: [PATCH 13/52] Addressed PR comment --- .../src/main/java/com/google/cloud/pubsub/v1/Publisher.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 454729f71..28135215a 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -91,6 +91,8 @@ public class Publisher implements PublisherInterface { private static final Logger logger = Logger.getLogger(Publisher.class.getName()); + private static final String GZIP_COMPRESSION = "gzip"; + private final String topicName; private final BatchingSettings batchingSettings; @@ -446,7 +448,7 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { private ApiFuture publishCall(OutstandingBatch outstandingBatch) { if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { GrpcCallContext context = GrpcCallContext.createDefault(); - context = context.withCallOptions(CallOptions.DEFAULT.withCompression("gzip")); + context = context.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); return publisherStub .publishCallable() .futureCall( From 4bbe64a84437f77348a2507908cc571f1ac55199 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 10 Mar 2022 16:02:33 -0500 Subject: [PATCH 14/52] Addressing PR comment to put a Precondition for compression and its threshold --- .../com/google/cloud/pubsub/v1/Publisher.java | 15 +++++++---- .../cloud/pubsub/v1/PublisherImplTest.java | 25 ++++++++++++++++++- 2 files changed, 34 insertions(+), 6 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 28135215a..0aa530529 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -119,6 +119,7 @@ public class Publisher implements PublisherInterface { private MessageFlowController flowController = null; private final boolean enableCompression; + private final boolean enableCompressionBytesThreshold; private final long compressionBytesThreshold; /** The maximum number of messages in one request. Defined by the API. */ @@ -148,6 +149,7 @@ private Publisher(Builder builder) throws IOException { this.enableMessageOrdering = builder.enableMessageOrdering; this.messageTransform = builder.messageTransform; this.enableCompression = builder.enableCompression; + this.enableCompressionBytesThreshold = builder.enableCompressionBytesThreshold; this.compressionBytesThreshold = builder.compressionBytesThreshold; messagesBatches = new HashMap<>(); @@ -212,11 +214,6 @@ public String getTopicNameString() { return topicName; } - /** Returns True/False if compression is enabled or disabled respectively. */ - public boolean getEnableCompression() { - return this.enableCompression; - } - /** * Schedules the publishing of a message. The publishing of the message may occur immediately or * be delayed based on the publisher batching options. @@ -256,6 +253,12 @@ public ApiFuture publish(PubsubMessage message) { + "Publisher client. Please create a Publisher client with " + "setEnableMessageOrdering(true) in the builder."); + Preconditions.checkState( + !enableCompressionBytesThreshold || enableCompression, + "Cannot publish a message with compression bytes threshold when compression is not enabled " + + "in the Publisher client. Please create a Publisher client with " + + "setEnableCompression(true) in the builder."); + final OutstandingPublish outstandingPublish = new OutstandingPublish(messageTransform.apply(message)); @@ -748,6 +751,7 @@ public PubsubMessage apply(PubsubMessage input) { }; private boolean enableCompression = DEFAULT_ENABLE_COMPRESSION; + private boolean enableCompressionBytesThreshold = DEFAULT_ENABLE_COMPRESSION; private long compressionBytesThreshold = DEFAULT_COMPRESSION_BYTES_THRESHOLD; private Builder(String topic) { @@ -873,6 +877,7 @@ public Builder setEnableCompression(boolean enableCompression) { * with this method. */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { + this.enableCompressionBytesThreshold = true; this.compressionBytesThreshold = compressionBytesThreshold; return this; } diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java index 1374ffad4..f82530bea 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java @@ -293,8 +293,8 @@ public void testPublishWithCompression() throws Exception { .setDelayThreshold(Duration.ofSeconds(100)) .build()) .setEnableCompression(true) + .setCompressionBytesThreshold(100) .build(); - assertTrue(publisher.getEnableCompression()); testPublisherServiceImpl.addPublishResponse( PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); @@ -307,6 +307,29 @@ public void testPublishWithCompression() throws Exception { shutdownTestPublisher(publisher); } + @Test + public void testCompressionBytesThresholdWhenCompressionDisabled_throwsException() + throws Exception { + // Compression is not enabled but compression bytes theshold is set + Publisher publisher = + getTestPublisherBuilder() + .setBatchingSettings( + Publisher.Builder.DEFAULT_BATCHING_SETTINGS + .toBuilder() + .setElementCountThreshold(2L) + .setDelayThreshold(Duration.ofSeconds(100)) + .build()) + .setCompressionBytesThreshold(200) + .build(); + try { + ApiFuture publishFuture = sendTestMessage(publisher, "A"); + fail("Should have thrown an IllegalStateException"); + } catch (IllegalStateException expected) { + // expected + } + shutdownTestPublisher(publisher); + } + private ApiFuture sendTestMessage(Publisher publisher, String data) { return publisher.publish( PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(data)).build()); From 3a8c5f70c27b9b7f2e5c00967ff0c26e418c88b0 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 11 Mar 2022 15:33:54 -0500 Subject: [PATCH 15/52] Addressing PR review --- .../com/google/cloud/pubsub/v1/Publisher.java | 7 +++---- .../pubsub/PublishWithCompressionExample.java | 18 ++++-------------- 2 files changed, 7 insertions(+), 18 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 0aa530529..5d01cad2f 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -865,16 +865,15 @@ public Builder setEndpoint(String endpoint) { return this; } - /** Gives the ability to enable gRPC compression. */ + /** Gives the ability to enable transport compression. */ public Builder setEnableCompression(boolean enableCompression) { this.enableCompression = enableCompression; return this; } /** - * Gives the ability to set the threshold in bytes above which gRPC compression happens. But to - * enable gRPC compression at the first place, setEnableCompression(true) should be called along - * with this method. + * Sets the threshold (in bytes) above which messages are compressed for transport. + * Only takes effect if setEnableCompression(true) is also called." */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { this.enableCompressionBytesThreshold = true; diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index e625e335d..71eb83679 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -1,5 +1,5 @@ /* - * Copyright 2020 Google LLC + * Copyright 2022 Google LLC * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -54,10 +54,11 @@ public static void publishWithCompressionExample(String projectId, String topicI publisher = Publisher.newBuilder(topicName) .setEnableCompression(true) - .setCompressionBytesThreshold(500) + .setCompressionBytesThreshold(10) .build(); - ByteString data = generateData("Hello!", 2000); + String message = "Hello World!"; + ByteString data = ByteString.copyFromUtf8(message); PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); // Once published, returns a server-assigned message id (unique within the topic) @@ -73,17 +74,6 @@ public static void publishWithCompressionExample(String projectId, String topicI } } - /** Generates data of given bytes by repeatedly concatenating a token. */ - // TODO(developer): Replace this method with your own data generation logic - private static ByteString generateData(String token, int bytes) { - String message = ""; - int tokenBytes = token.length(); - for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { - message = message.concat(token); - } - return ByteString.copyFromUtf8(message); - } - /** * Sets up logging to observe the outbound data (and its length) over the network to analyze the * effectiveness of compression. A sample log line: From 01c7e3f24d86e5304f67f23c76de8f65484c95f0 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 11 Mar 2022 16:52:23 -0500 Subject: [PATCH 16/52] Removing logging from example --- .../pubsub/PublishWithCompressionExample.java | 32 ------------------- 1 file changed, 32 deletions(-) diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 71eb83679..7457c8c5d 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -35,11 +35,7 @@ public static void main(String... args) throws Exception { // TODO(developer): Replace these variables before running the sample. String projectId = "your-project-id"; String topicId = "your-topic-id"; - boolean allowLogging = false; // Set to true to get the stdout logs - if (allowLogging) { - setUpLogs(); - } publishWithCompressionExample(projectId, topicId); } @@ -73,33 +69,5 @@ public static void publishWithCompressionExample(String projectId, String topicI } } } - - /** - * Sets up logging to observe the outbound data (and its length) over the network to analyze the - * effectiveness of compression. A sample log line: - * [2022-03-02] FINE [] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01.. - */ - private static void setUpLogs() throws IOException { - String handlers = "handlers = java.util.logging.ConsoleHandler"; - String handlerLevelProp = "java.util.logging.ConsoleHandler.level = ALL"; - String fineProp = ".level = FINE"; - String handlerFormatterProp = - "java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter"; - String format = "java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n"; - - LogManager.getLogManager() - .readConfiguration( - new ByteArrayInputStream( - (handlers - + "\n" - + handlerLevelProp - + "\n" - + fineProp - + "\n" - + handlerFormatterProp - + "\n" - + format) - .getBytes(StandardCharsets.UTF_8))); - } } // [END pubsub_publish_with_compression] From 22dee27116b3fdca2c09243b4c520179c6ebe812 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 11 Mar 2022 16:53:22 -0500 Subject: [PATCH 17/52] Adding logging properties --- samples/snippets/src/main/resources/logging.properties | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 samples/snippets/src/main/resources/logging.properties diff --git a/samples/snippets/src/main/resources/logging.properties b/samples/snippets/src/main/resources/logging.properties new file mode 100644 index 000000000..1f482dad2 --- /dev/null +++ b/samples/snippets/src/main/resources/logging.properties @@ -0,0 +1,5 @@ +handlers = java.util.logging.ConsoleHandler +java.util.logging.ConsoleHandler.level = ALL +.level = FINE +java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter +java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n \ No newline at end of file From 78325994ca58aec459a467043c1ce21e3bc201f0 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Tue, 15 Mar 2022 14:04:35 -0400 Subject: [PATCH 18/52] Making the publish call unified with context as per PR comments --- .../com/google/cloud/pubsub/v1/Publisher.java | 30 +++++++------------ 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 5d01cad2f..c65bfca92 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -449,26 +449,18 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { + GrpcCallContext context = GrpcCallContext.createDefault(); if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { - GrpcCallContext context = GrpcCallContext.createDefault(); context = context.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); - return publisherStub - .publishCallable() - .futureCall( - PublishRequest.newBuilder() - .setTopic(topicName) - .addAllMessages(outstandingBatch.getMessages()) - .build(), - context); - } else { - return publisherStub - .publishCallable() - .futureCall( - PublishRequest.newBuilder() - .setTopic(topicName) - .addAllMessages(outstandingBatch.getMessages()) - .build()); } + return publisherStub + .publishCallable() + .futureCall( + PublishRequest.newBuilder() + .setTopic(topicName) + .addAllMessages(outstandingBatch.getMessages()) + .build(), + context); } private void publishOutstandingBatch(final OutstandingBatch outstandingBatch) { @@ -872,8 +864,8 @@ public Builder setEnableCompression(boolean enableCompression) { } /** - * Sets the threshold (in bytes) above which messages are compressed for transport. - * Only takes effect if setEnableCompression(true) is also called." + * Sets the threshold (in bytes) above which messages are compressed for transport. Only takes + * effect if setEnableCompression(true) is also called." */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { this.enableCompressionBytesThreshold = true; From 9f3fcc0d193fdf25ac629a2a362506924f250497 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 21 Mar 2022 16:17:41 -0400 Subject: [PATCH 19/52] Removing sample code as per tianzi@'s comments --- README.md | 1 - .../pubsub/PublishWithCompressionExample.java | 73 ------------------- .../src/main/resources/logging.properties | 5 -- 3 files changed, 79 deletions(-) delete mode 100644 samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java delete mode 100644 samples/snippets/src/main/resources/logging.properties diff --git a/README.md b/README.md index 13bad9a07..382a5fe8f 100644 --- a/README.md +++ b/README.md @@ -272,7 +272,6 @@ Samples are in the [`samples/`](https://github.com/googleapis/java-pubsub/tree/m | Publish With Flow Control Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | | Publish With Ordering Keys | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | | Publish With Retry Settings Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | -| Publish With Compression Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | | Publisher Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublisherExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublisherExample.java) | | Receive Messages With Delivery Attempts Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | | Remove Dead Letter Policy Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java deleted file mode 100644 index 7457c8c5d..000000000 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed 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 pubsub; - -// [START pubsub_publish_with_compression] -import com.google.api.core.ApiFuture; -import com.google.cloud.pubsub.v1.Publisher; -import com.google.protobuf.ByteString; -import com.google.pubsub.v1.PubsubMessage; -import com.google.pubsub.v1.TopicName; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.logging.LogManager; - -public class PublishWithCompressionExample { - - public static void main(String... args) throws Exception { - // TODO(developer): Replace these variables before running the sample. - String projectId = "your-project-id"; - String topicId = "your-topic-id"; - - publishWithCompressionExample(projectId, topicId); - } - - public static void publishWithCompressionExample(String projectId, String topicId) - throws IOException, ExecutionException, InterruptedException { - TopicName topicName = TopicName.of(projectId, topicId); - - Publisher publisher = null; - try { - // Create a publisher instance bound to the topic with compression enabled and a compression - // bytes threshold. - publisher = - Publisher.newBuilder(topicName) - .setEnableCompression(true) - .setCompressionBytesThreshold(10) - .build(); - - String message = "Hello World!"; - ByteString data = ByteString.copyFromUtf8(message); - PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); - - // Once published, returns a server-assigned message id (unique within the topic) - ApiFuture messageIdFuture = publisher.publish(pubsubMessage); - String messageId = messageIdFuture.get(); - System.out.println("Published compressed message, ID:" + messageId); - } finally { - if (publisher != null) { - // When finished with the publisher, shutdown to free up resources. - publisher.shutdown(); - publisher.awaitTermination(1, TimeUnit.MINUTES); - } - } - } -} -// [END pubsub_publish_with_compression] diff --git a/samples/snippets/src/main/resources/logging.properties b/samples/snippets/src/main/resources/logging.properties deleted file mode 100644 index 1f482dad2..000000000 --- a/samples/snippets/src/main/resources/logging.properties +++ /dev/null @@ -1,5 +0,0 @@ -handlers = java.util.logging.ConsoleHandler -java.util.logging.ConsoleHandler.level = ALL -.level = FINE -java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter -java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n \ No newline at end of file From 9ae5944896410621a73efc8de431be8ed3bde5eb Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 11 Apr 2022 11:45:37 -0400 Subject: [PATCH 20/52] Minor fixes --- .../com/google/cloud/pubsub/v1/Publisher.java | 2 +- .../google/cloud/pubsub/it/ITPubSubTest.java | 20 ++++++------------- .../src/test/java/pubsub/PublisherIT.java | 5 ----- 3 files changed, 7 insertions(+), 20 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index c65bfca92..06ff3d767 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -712,7 +712,7 @@ public static final class Builder { .setExecutorThreadCount(THREADS_PER_CPU * Runtime.getRuntime().availableProcessors()) .build(); static final boolean DEFAULT_ENABLE_COMPRESSION = false; - static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; + static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 240L; String topicName; private String endpoint = PublisherStubSettings.getDefaultEndpoint(); diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index c5e512e65..8d61ab6c5 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -24,14 +24,7 @@ import com.google.api.gax.rpc.PermissionDeniedException; import com.google.auto.value.AutoValue; import com.google.cloud.ServiceOptions; -import com.google.cloud.pubsub.v1.AckReplyConsumer; -import com.google.cloud.pubsub.v1.MessageReceiver; -import com.google.cloud.pubsub.v1.Publisher; -import com.google.cloud.pubsub.v1.Subscriber; -import com.google.cloud.pubsub.v1.SubscriptionAdminClient; -import com.google.cloud.pubsub.v1.SubscriptionAdminSettings; -import com.google.cloud.pubsub.v1.TopicAdminClient; -import com.google.cloud.pubsub.v1.TopicAdminSettings; +import com.google.cloud.pubsub.v1.*; import com.google.common.util.concurrent.MoreExecutors; import com.google.iam.v1.Binding; import com.google.iam.v1.GetIamPolicyRequest; @@ -90,9 +83,8 @@ static MessageAndConsumerWithResponse create( @BeforeClass public static void setupClass() throws Exception { - topicAdminClient = TopicAdminClient.create(TopicAdminSettings.newBuilder().build()); - subscriptionAdminClient = - SubscriptionAdminClient.create(SubscriptionAdminSettings.newBuilder().build()); + topicAdminClient = TopicAdminClient.create(); + subscriptionAdminClient = SubscriptionAdminClient.create(); projectId = ServiceOptions.getDefaultProjectId(); } @@ -420,7 +412,7 @@ public void testPublishSubscribeWithCompression() throws Exception { topicAdminClient.createTopic(topicName); subscriptionAdminClient.createSubscription( - getSubscription(subscriptionName, topicName, PushConfig.newBuilder().build(), 10)); + getSubscription(subscriptionName, topicName, PushConfig.newBuilder().build(), 10, false)); final BlockingQueue receiveQueue = new LinkedBlockingQueue<>(); Subscriber subscriber = @@ -457,12 +449,12 @@ public void failed(Subscriber.State from, Throwable failure) { publisher.awaitTermination(1, TimeUnit.MINUTES); // Ack the first message. - MessageAndConsumer toAck1 = pollQueue(receiveQueue); + MessageAndConsumer toAck1 = pollQueueMessageAndConsumer(receiveQueue); assertThat(toAck1.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg1)); toAck1.consumer().ack(); // Ack the second message. - MessageAndConsumer toAck2 = pollQueue(receiveQueue); + MessageAndConsumer toAck2 = pollQueueMessageAndConsumer(receiveQueue); assertThat(toAck2.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg2)); toAck2.consumer().ack(); diff --git a/samples/snippets/src/test/java/pubsub/PublisherIT.java b/samples/snippets/src/test/java/pubsub/PublisherIT.java index 48525e7d6..b22912d47 100644 --- a/samples/snippets/src/test/java/pubsub/PublisherIT.java +++ b/samples/snippets/src/test/java/pubsub/PublisherIT.java @@ -124,10 +124,5 @@ public void testPublisher() throws Exception { for (int i = 1; i <= 4; i++) { assertThat(bout.toString()).contains("message" + i); } - - bout.reset(); - // Test publish with compression. - PublishWithCompressionExample.publishWithCompressionExample(projectId, topicId); - assertThat(bout.toString()).contains("Published compressed message ID: "); } } From 2fa8849bc0fd4e7b211262c80d7c9a0d8fc0e524 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 3 Feb 2022 11:09:16 -0800 Subject: [PATCH 21/52] Adding gRPC compression support to the library --- .../com/google/cloud/pubsub/v1/Publisher.java | 44 ++++++++++++++++--- 1 file changed, 37 insertions(+), 7 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 39302d3f7..25670ec49 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -35,6 +35,7 @@ import com.google.api.gax.core.ExecutorProvider; import com.google.api.gax.core.FixedExecutorProvider; import com.google.api.gax.core.InstantiatingExecutorProvider; +import com.google.api.gax.grpc.GrpcCallContext; import com.google.api.gax.retrying.RetrySettings; import com.google.api.gax.rpc.HeaderProvider; import com.google.api.gax.rpc.NoHeaderProvider; @@ -67,6 +68,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.logging.Level; import java.util.logging.Logger; +import io.grpc.CallOptions; import org.threeten.bp.Duration; /** @@ -114,6 +116,11 @@ public class Publisher implements PublisherInterface { private MessageFlowController flowController = null; + private final boolean enableCompression; + + /** The message is compressed when its size (in bytes) is above the threshold. */ + private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte (https://en.wikipedia.org/wiki/Kilobyte) + /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { return 1000L; @@ -191,6 +198,8 @@ private Publisher(Builder builder) throws IOException { backgroundResources = new BackgroundResourceAggregation(backgroundResourceList); shutdown = new AtomicBoolean(false); messagesWaiter = new Waiter(); + + enableCompression = builder.enableCompression; } /** Topic which the publisher publishes to. */ @@ -431,13 +440,25 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { - return publisherStub - .publishCallable() - .futureCall( - PublishRequest.newBuilder() - .setTopic(topicName) - .addAllMessages(outstandingBatch.getMessages()) - .build()); + if (enableCompression && outstandingBatch.batchSizeBytes >= MSG_COMPRESSION_THRESHOLD_BYTES) { + GrpcCallContext context = GrpcCallContext.createDefault(); + context = context.withCallOptions(CallOptions.DEFAULT.withCompression("gzip")); + return publisherStub + .publishCallable() + .futureCall( + PublishRequest.newBuilder() + .setTopic(topicName) + .addAllMessages(outstandingBatch.getMessages()) + .build(), context); + } else { + return publisherStub + .publishCallable() + .futureCall( + PublishRequest.newBuilder() + .setTopic(topicName) + .addAllMessages(outstandingBatch.getMessages()) + .build()); + } } private void publishOutstandingBatch(final OutstandingBatch outstandingBatch) { @@ -688,6 +709,7 @@ public static final class Builder { InstantiatingExecutorProvider.newBuilder() .setExecutorThreadCount(THREADS_PER_CPU * Runtime.getRuntime().availableProcessors()) .build(); + static final boolean DEFAULT_ENABLE_COMPRESSION = false; String topicName; private String endpoint = PublisherStubSettings.getDefaultEndpoint(); @@ -717,6 +739,8 @@ public PubsubMessage apply(PubsubMessage input) { } }; + private boolean enableCompression = DEFAULT_ENABLE_COMPRESSION; + private Builder(String topic) { this.topicName = Preconditions.checkNotNull(topic); } @@ -827,6 +851,12 @@ public Builder setEndpoint(String endpoint) { return this; } + /** Gives the ability to enable gRPC compression. */ + public Builder setEnableCompression(boolean enableCompression) { + this.enableCompression = enableCompression; + return this; + } + /** Returns the default BatchingSettings used by the client if settings are not provided. */ public static BatchingSettings getDefaultBatchingSettings() { return DEFAULT_BATCHING_SETTINGS; From ab3fc37410e71619b2a932d40fb991606e79ff8c Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 3 Feb 2022 11:11:23 -0800 Subject: [PATCH 22/52] Minor comment fix --- .../src/main/java/com/google/cloud/pubsub/v1/Publisher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 25670ec49..b42c55275 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -119,7 +119,7 @@ public class Publisher implements PublisherInterface { private final boolean enableCompression; /** The message is compressed when its size (in bytes) is above the threshold. */ - private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte (https://en.wikipedia.org/wiki/Kilobyte) + private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { From d71a05205f662955507bd75f1d0eee2618fe1e13 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 3 Feb 2022 22:55:30 +0000 Subject: [PATCH 23/52] Formatting the code --- .../main/java/com/google/cloud/pubsub/v1/Publisher.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index b42c55275..8a897d1f2 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -51,6 +51,7 @@ import com.google.pubsub.v1.PubsubMessage; import com.google.pubsub.v1.TopicName; import com.google.pubsub.v1.TopicNames; +import io.grpc.CallOptions; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -68,7 +69,6 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.logging.Level; import java.util.logging.Logger; -import io.grpc.CallOptions; import org.threeten.bp.Duration; /** @@ -449,9 +449,10 @@ private ApiFuture publishCall(OutstandingBatch outstandingBatch PublishRequest.newBuilder() .setTopic(topicName) .addAllMessages(outstandingBatch.getMessages()) - .build(), context); - } else { - return publisherStub + .build(), + context); + } else { + return publisherStub .publishCallable() .futureCall( PublishRequest.newBuilder() From 99686ea4003a5b9a0348c1d1ece0443b06ad8b5d Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 7 Feb 2022 18:40:16 -0500 Subject: [PATCH 24/52] Adding unit test for compression --- .../com/google/cloud/pubsub/v1/Publisher.java | 8 ++++-- .../cloud/pubsub/v1/PublisherImplTest.java | 25 +++++++++++++++++++ 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 8a897d1f2..b6bb61ab2 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -147,6 +147,7 @@ private Publisher(Builder builder) throws IOException { this.enableMessageOrdering = builder.enableMessageOrdering; this.messageTransform = builder.messageTransform; + this.enableCompression = builder.enableCompression; messagesBatches = new HashMap<>(); messagesBatchLock = new ReentrantLock(); @@ -198,8 +199,6 @@ private Publisher(Builder builder) throws IOException { backgroundResources = new BackgroundResourceAggregation(backgroundResourceList); shutdown = new AtomicBoolean(false); messagesWaiter = new Waiter(); - - enableCompression = builder.enableCompression; } /** Topic which the publisher publishes to. */ @@ -212,6 +211,11 @@ public String getTopicNameString() { return topicName; } + /** Returns True/False if compression is enabled or disabled respectively. */ + public boolean getEnableCompression() { + return this.enableCompression; + } + /** * Schedules the publishing of a message. The publishing of the message may occur immediately or * be delayed based on the publisher batching options. diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java index c41931de6..9a8ee9494 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java @@ -282,6 +282,31 @@ public void testPublishMixedSizeAndDuration() throws Exception { shutdownTestPublisher(publisher); } + @Test + public void testPublishWithCompression() throws Exception { + Publisher publisher = + getTestPublisherBuilder() + .setBatchingSettings( + Publisher.Builder.DEFAULT_BATCHING_SETTINGS + .toBuilder() + .setElementCountThreshold(2L) + .setDelayThreshold(Duration.ofSeconds(100)) + .build()) + .setEnableCompression(true) + .build(); + assertTrue(publisher.getEnableCompression()); + + testPublisherServiceImpl + .addPublishResponse(PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); + ApiFuture publishFuture1 = sendTestMessage(publisher, "A"); + ApiFuture publishFuture2 = sendTestMessage(publisher, "B"); + assertEquals("1", publishFuture1.get()); + assertEquals("2", publishFuture2.get()); + + fakeExecutor.advanceTime(Duration.ofSeconds(100)); + shutdownTestPublisher(publisher); + } + private ApiFuture sendTestMessage(Publisher publisher, String data) { return publisher.publish( PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(data)).build()); From d8816caf54fc753fff141000bfb4c00d0a1e4326 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 7 Feb 2022 21:01:33 -0500 Subject: [PATCH 25/52] Adding integration test for compression --- .../google/cloud/pubsub/it/ITPubSubTest.java | 74 +++++++++++++++++++ .../cloud/pubsub/v1/PublisherImplTest.java | 4 +- 2 files changed, 76 insertions(+), 2 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index c72d52d3d..d25156bf4 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -403,6 +403,70 @@ public void failed(Subscriber.State from, Throwable failure) { topicAdminClient.deleteTopic(topicName); } + @Test + public void testPublishSubscribeWithCompression() throws Exception { + TopicName topicName = + TopicName.newBuilder() + .setProject(projectId) + .setTopic(formatForTest("testing-compression-topic")) + .build(); + SubscriptionName subscriptionName = + SubscriptionName.of(projectId, formatForTest("testing-compression-subscription")); + + topicAdminClient.createTopic(topicName); + + subscriptionAdminClient.createSubscription( + getSubscription(subscriptionName, topicName, PushConfig.newBuilder().build(), 10)); + + final BlockingQueue receiveQueue = new LinkedBlockingQueue<>(); + Subscriber subscriber = + Subscriber.newBuilder( + subscriptionName.toString(), + new MessageReceiver() { + @Override + public void receiveMessage( + final PubsubMessage message, final AckReplyConsumer consumer) { + receiveQueue.offer(MessageAndConsumer.create(message, consumer)); + } + }) + .build(); + subscriber.addListener( + new Subscriber.Listener() { + public void failed(Subscriber.State from, Throwable failure) { + receiveQueue.offer(failure); + } + }, + MoreExecutors.directExecutor()); + subscriber.startAsync(); + + Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + + String msg1 = generateMessage("msg1", 1000); + String msg2 = generateMessage("msg2", 1500); + publisher + .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(msg1)).build()) + .get(); + publisher + .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(msg2)).build()) + .get(); + publisher.shutdown(); + publisher.awaitTermination(1, TimeUnit.MINUTES); + + // Ack the first message. + MessageAndConsumer toAck1 = pollQueue(receiveQueue); + assertThat(toAck1.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg1)); + toAck1.consumer().ack(); + + // Ack the second message. + MessageAndConsumer toAck2 = pollQueue(receiveQueue); + assertThat(toAck2.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg2)); + toAck2.consumer().ack(); + + subscriber.stopAsync().awaitTerminated(); + subscriptionAdminClient.deleteSubscription(subscriptionName); + topicAdminClient.deleteTopic(topicName); + } + private MessageAndConsumer pollQueueMessageAndConsumer(BlockingQueue queue) throws InterruptedException { Object obj = pollQueue(queue); @@ -434,4 +498,14 @@ private Object pollQueue(BlockingQueue queue) throws InterruptedExceptio return obj; } + + /** Generates message of given bytes by repeatedly concatenating a token. */ + private String generateMessage(String token, int bytes) { + String result = ""; + int tokenBytes = token.length(); + for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { + result = result.concat(token); + } + return result; + } } diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java index 9a8ee9494..1374ffad4 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java @@ -296,8 +296,8 @@ public void testPublishWithCompression() throws Exception { .build(); assertTrue(publisher.getEnableCompression()); - testPublisherServiceImpl - .addPublishResponse(PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); + testPublisherServiceImpl.addPublishResponse( + PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); ApiFuture publishFuture1 = sendTestMessage(publisher, "A"); ApiFuture publishFuture2 = sendTestMessage(publisher, "B"); assertEquals("1", publishFuture1.get()); From cfb85a0e9e435f9b3a5b5fa685d329290eedb3fd Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Tue, 8 Feb 2022 12:09:37 -0500 Subject: [PATCH 26/52] Formatting --- .../java/com/google/cloud/pubsub/it/ITPubSubTest.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index d25156bf4..afc94d42b 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -40,11 +40,7 @@ import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; +import org.junit.*; import org.junit.rules.Timeout; public class ITPubSubTest { @@ -439,7 +435,10 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + Publisher publisher = + Publisher.newBuilder(topicName) + .setEnableCompression(true) + .build(); String msg1 = generateMessage("msg1", 1000); String msg2 = generateMessage("msg2", 1500); From 1d89073dae8be560c9b1448a14c6dd02c54b668a Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Tue, 8 Feb 2022 15:02:47 -0500 Subject: [PATCH 27/52] Refactoring integration tests to add support for overriding endpoint --- .../google/cloud/pubsub/it/ITPubSubTest.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index afc94d42b..cf2026d41 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -24,7 +24,14 @@ import com.google.api.gax.rpc.PermissionDeniedException; import com.google.auto.value.AutoValue; import com.google.cloud.ServiceOptions; -import com.google.cloud.pubsub.v1.*; +import com.google.cloud.pubsub.v1.AckReplyConsumer; +import com.google.cloud.pubsub.v1.MessageReceiver; +import com.google.cloud.pubsub.v1.Publisher; +import com.google.cloud.pubsub.v1.Subscriber; +import com.google.cloud.pubsub.v1.SubscriptionAdminClient; +import com.google.cloud.pubsub.v1.SubscriptionAdminSettings; +import com.google.cloud.pubsub.v1.TopicAdminClient; +import com.google.cloud.pubsub.v1.TopicAdminSettings; import com.google.common.util.concurrent.MoreExecutors; import com.google.iam.v1.Binding; import com.google.iam.v1.GetIamPolicyRequest; @@ -49,6 +56,7 @@ public class ITPubSubTest { private static TopicAdminClient topicAdminClient; private static SubscriptionAdminClient subscriptionAdminClient; private static String projectId; + private static String endPoint = "us-east1-pubsub.googleapis.com:443"; private static final boolean IS_VPC_TEST = System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC") != null && System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC").equalsIgnoreCase("true"); @@ -83,8 +91,11 @@ static MessageAndConsumerWithResponse create( @BeforeClass public static void setupClass() throws Exception { - topicAdminClient = TopicAdminClient.create(); - subscriptionAdminClient = SubscriptionAdminClient.create(); + topicAdminClient = + TopicAdminClient.create(TopicAdminSettings.newBuilder().setEndpoint(endPoint).build()); + subscriptionAdminClient = + SubscriptionAdminClient.create( + SubscriptionAdminSettings.newBuilder().setEndpoint(endPoint).build()); projectId = ServiceOptions.getDefaultProjectId(); } @@ -205,6 +216,7 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) + .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -215,7 +227,7 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = Publisher.newBuilder(topicName).build(); + Publisher publisher = Publisher.newBuilder(topicName).setEndpoint(endPoint).build(); publisher .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8("msg1")).build()) .get(); @@ -425,6 +437,7 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) + .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -436,9 +449,7 @@ public void failed(Subscriber.State from, Throwable failure) { subscriber.startAsync(); Publisher publisher = - Publisher.newBuilder(topicName) - .setEnableCompression(true) - .build(); + Publisher.newBuilder(topicName).setEndpoint(endPoint).setEnableCompression(true).build(); String msg1 = generateMessage("msg1", 1000); String msg2 = generateMessage("msg2", 1500); From ed13022d0c863819d684ba64f35e58ce48b69369 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 9 Feb 2022 17:01:03 -0500 Subject: [PATCH 28/52] Adding sample for publish with compression; Updating README --- README.md | 1 + .../pubsub/PublishWithCompressionExample.java | 76 +++++++++++++++++++ 2 files changed, 77 insertions(+) create mode 100644 samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java diff --git a/README.md b/README.md index 593f91dec..4575723d9 100644 --- a/README.md +++ b/README.md @@ -272,6 +272,7 @@ Samples are in the [`samples/`](https://github.com/googleapis/java-pubsub/tree/m | Publish With Flow Control Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | | Publish With Ordering Keys | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | | Publish With Retry Settings Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | +| Publish With Compression Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | | Publisher Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublisherExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublisherExample.java) | | Receive Messages With Delivery Attempts Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | | Remove Dead Letter Policy Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java new file mode 100644 index 000000000..da2e5c318 --- /dev/null +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -0,0 +1,76 @@ +/* + * Copyright 2020 Google LLC + * + * Licensed 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 pubsub; + +// [START pubsub_publish_with_compression] +import com.google.api.core.ApiFuture; +import com.google.cloud.pubsub.v1.Publisher; +import com.google.protobuf.ByteString; +import com.google.pubsub.v1.PubsubMessage; +import com.google.pubsub.v1.TopicName; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +public class PublishWithCompressionExample { + public static void main(String... args) throws Exception { + // TODO(developer): Replace these variables before running the sample. + String projectId = "your-project-id"; + // Choose an existing topic. + String topicId = "your-topic-id"; + + publishWithCompression(projectId, topicId); + } + + public static void publishWithCompression(String projectId, String topicId) + throws IOException, ExecutionException, InterruptedException { + TopicName topicName = TopicName.of(projectId, topicId); + + Publisher publisher = null; + try { + // Create a publisher instance with default settings bound to the topic + publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + + // Compression works only for messages of size >= 1KB + String message = generateMessage("Hello!", 2000); + ByteString data = ByteString.copyFromUtf8(message); + PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); + + // Once published, returns a server-assigned message id (unique within the topic) + ApiFuture messageIdFuture = publisher.publish(pubsubMessage); + String messageId = messageIdFuture.get(); + System.out.println("Published message ID: " + messageId); + } finally { + if (publisher != null) { + // When finished with the publisher, shutdown to free up resources. + publisher.shutdown(); + publisher.awaitTermination(1, TimeUnit.MINUTES); + } + } + } + + /** Generate message of given bytes by repeatedly concatenating a token.* */ + private static String generateMessage(String token, int bytes) { + String result = ""; + int tokenBytes = token.length(); + for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { + result = result.concat(token); + } + return result; + } +} +// [END pubsub_publish_with_compression] From a133b99355add3a9121c90a27bb1718fe7167353 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 9 Feb 2022 17:48:57 -0500 Subject: [PATCH 29/52] Adding integration test for compression sample --- .../src/main/java/pubsub/PublishWithCompressionExample.java | 6 +++--- samples/snippets/src/test/java/pubsub/PublisherIT.java | 5 +++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index da2e5c318..40273e2f8 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -33,10 +33,10 @@ public static void main(String... args) throws Exception { // Choose an existing topic. String topicId = "your-topic-id"; - publishWithCompression(projectId, topicId); + publishWithCompressionExample(projectId, topicId); } - public static void publishWithCompression(String projectId, String topicId) + public static void publishWithCompressionExample(String projectId, String topicId) throws IOException, ExecutionException, InterruptedException { TopicName topicName = TopicName.of(projectId, topicId); @@ -53,7 +53,7 @@ public static void publishWithCompression(String projectId, String topicId) // Once published, returns a server-assigned message id (unique within the topic) ApiFuture messageIdFuture = publisher.publish(pubsubMessage); String messageId = messageIdFuture.get(); - System.out.println("Published message ID: " + messageId); + System.out.println("Published compressed message ID: " + messageId); } finally { if (publisher != null) { // When finished with the publisher, shutdown to free up resources. diff --git a/samples/snippets/src/test/java/pubsub/PublisherIT.java b/samples/snippets/src/test/java/pubsub/PublisherIT.java index b22912d47..48525e7d6 100644 --- a/samples/snippets/src/test/java/pubsub/PublisherIT.java +++ b/samples/snippets/src/test/java/pubsub/PublisherIT.java @@ -124,5 +124,10 @@ public void testPublisher() throws Exception { for (int i = 1; i <= 4; i++) { assertThat(bout.toString()).contains("message" + i); } + + bout.reset(); + // Test publish with compression. + PublishWithCompressionExample.publishWithCompressionExample(projectId, topicId); + assertThat(bout.toString()).contains("Published compressed message ID: "); } } From 9087f22baccbee46b08e6701c8ef5ae833c24c6e Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 16 Feb 2022 11:55:35 -0500 Subject: [PATCH 30/52] Adding parameter compressionBytesThreshold to Publisher; Adding logging support in the compression example --- .../com/google/cloud/pubsub/v1/Publisher.java | 15 ++++-- .../pubsub/PublishWithCompressionExample.java | 50 +++++++++++++++---- 2 files changed, 51 insertions(+), 14 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index b6bb61ab2..c29cfdefd 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -117,9 +117,7 @@ public class Publisher implements PublisherInterface { private MessageFlowController flowController = null; private final boolean enableCompression; - - /** The message is compressed when its size (in bytes) is above the threshold. */ - private static final long MSG_COMPRESSION_THRESHOLD_BYTES = 1000L; // 1 kilobyte + private final long compressionBytesThreshold; /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { @@ -148,6 +146,7 @@ private Publisher(Builder builder) throws IOException { this.enableMessageOrdering = builder.enableMessageOrdering; this.messageTransform = builder.messageTransform; this.enableCompression = builder.enableCompression; + this.compressionBytesThreshold = builder.compressionBytesThreshold; messagesBatches = new HashMap<>(); messagesBatchLock = new ReentrantLock(); @@ -444,7 +443,7 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { - if (enableCompression && outstandingBatch.batchSizeBytes >= MSG_COMPRESSION_THRESHOLD_BYTES) { + if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { GrpcCallContext context = GrpcCallContext.createDefault(); context = context.withCallOptions(CallOptions.DEFAULT.withCompression("gzip")); return publisherStub @@ -681,6 +680,7 @@ public static final class Builder { // Meaningful defaults. static final long DEFAULT_ELEMENT_COUNT_THRESHOLD = 100L; static final long DEFAULT_REQUEST_BYTES_THRESHOLD = 1000L; // 1 kB + static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; static final Duration DEFAULT_DELAY_THRESHOLD = Duration.ofMillis(1); private static final Duration DEFAULT_INITIAL_RPC_TIMEOUT = Duration.ofSeconds(5); private static final Duration DEFAULT_MAX_RPC_TIMEOUT = Duration.ofSeconds(60); @@ -745,6 +745,7 @@ public PubsubMessage apply(PubsubMessage input) { }; private boolean enableCompression = DEFAULT_ENABLE_COMPRESSION; + private long compressionBytesThreshold = DEFAULT_COMPRESSION_BYTES_THRESHOLD; private Builder(String topic) { this.topicName = Preconditions.checkNotNull(topic); @@ -862,6 +863,12 @@ public Builder setEnableCompression(boolean enableCompression) { return this; } + /** Gives the ability to set the threshold in bytes above which gRPC compression happens. */ + public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { + this.compressionBytesThreshold = compressionBytesThreshold; + return this; + } + /** Returns the default BatchingSettings used by the client if settings are not provided. */ public static BatchingSettings getDefaultBatchingSettings() { return DEFAULT_BATCHING_SETTINGS; diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 40273e2f8..36eb03933 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -22,17 +22,24 @@ import com.google.protobuf.ByteString; import com.google.pubsub.v1.PubsubMessage; import com.google.pubsub.v1.TopicName; +import java.io.ByteArrayInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.logging.LogManager; public class PublishWithCompressionExample { + public static void main(String... args) throws Exception { // TODO(developer): Replace these variables before running the sample. String projectId = "your-project-id"; - // Choose an existing topic. String topicId = "your-topic-id"; + boolean allowLogging = false; // Set to true to get the stdout logs + if (allowLogging) { + setUpLogs(); + } publishWithCompressionExample(projectId, topicId); } @@ -42,12 +49,11 @@ public static void publishWithCompressionExample(String projectId, String topicI Publisher publisher = null; try { - // Create a publisher instance with default settings bound to the topic + // Create a publisher instance bound to the topic with compression enabled and other default + // settings publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); - // Compression works only for messages of size >= 1KB - String message = generateMessage("Hello!", 2000); - ByteString data = ByteString.copyFromUtf8(message); + ByteString data = generateData("Hello!", 2000); PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); // Once published, returns a server-assigned message id (unique within the topic) @@ -63,14 +69,38 @@ public static void publishWithCompressionExample(String projectId, String topicI } } - /** Generate message of given bytes by repeatedly concatenating a token.* */ - private static String generateMessage(String token, int bytes) { - String result = ""; + // Generates data of given bytes by repeatedly concatenating a token. + // TODO(developer): Replace this method with your own data generation logic + private static ByteString generateData(String token, int bytes) { + String message = ""; int tokenBytes = token.length(); for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { - result = result.concat(token); + message = message.concat(token); } - return result; + return ByteString.copyFromUtf8(message); + } + + private static void setUpLogs() throws IOException { + String handlers = "handlers = java.util.logging.ConsoleHandler"; + String handlerLevelProp = "java.util.logging.ConsoleHandler.level = ALL"; + String fineProp = ".level = FINE"; + String handlerFormatterProp = + "java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter"; + String format = "java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n"; + + LogManager.getLogManager() + .readConfiguration( + new ByteArrayInputStream( + (handlers + + "\n" + + handlerLevelProp + + "\n" + + fineProp + + "\n" + + handlerFormatterProp + + "\n" + + format) + .getBytes(StandardCharsets.UTF_8))); } } // [END pubsub_publish_with_compression] From 509193c73251af6fe0f0dcb76496fc4631d1f881 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 2 Mar 2022 12:28:41 -0500 Subject: [PATCH 31/52] Addressing PR comments --- .../com/google/cloud/pubsub/v1/Publisher.java | 8 ++++++-- .../google/cloud/pubsub/it/ITPubSubTest.java | 14 ++++---------- .../pubsub/PublishWithCompressionExample.java | 19 ++++++++++++++----- 3 files changed, 24 insertions(+), 17 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index c29cfdefd..3be7fd019 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -680,7 +680,6 @@ public static final class Builder { // Meaningful defaults. static final long DEFAULT_ELEMENT_COUNT_THRESHOLD = 100L; static final long DEFAULT_REQUEST_BYTES_THRESHOLD = 1000L; // 1 kB - static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; static final Duration DEFAULT_DELAY_THRESHOLD = Duration.ofMillis(1); private static final Duration DEFAULT_INITIAL_RPC_TIMEOUT = Duration.ofSeconds(5); private static final Duration DEFAULT_MAX_RPC_TIMEOUT = Duration.ofSeconds(60); @@ -715,6 +714,7 @@ public static final class Builder { .setExecutorThreadCount(THREADS_PER_CPU * Runtime.getRuntime().availableProcessors()) .build(); static final boolean DEFAULT_ENABLE_COMPRESSION = false; + static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; String topicName; private String endpoint = PublisherStubSettings.getDefaultEndpoint(); @@ -863,7 +863,11 @@ public Builder setEnableCompression(boolean enableCompression) { return this; } - /** Gives the ability to set the threshold in bytes above which gRPC compression happens. */ + /** + * Gives the ability to set the threshold in bytes above which gRPC compression happens. But to + * enable gRPC compression at the first place, setEnableCompression(true) should be called along + * with this method. + */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { this.compressionBytesThreshold = compressionBytesThreshold; return this; diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index cf2026d41..c5e512e65 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -56,7 +56,6 @@ public class ITPubSubTest { private static TopicAdminClient topicAdminClient; private static SubscriptionAdminClient subscriptionAdminClient; private static String projectId; - private static String endPoint = "us-east1-pubsub.googleapis.com:443"; private static final boolean IS_VPC_TEST = System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC") != null && System.getenv("GOOGLE_CLOUD_TESTS_IN_VPCSC").equalsIgnoreCase("true"); @@ -91,11 +90,9 @@ static MessageAndConsumerWithResponse create( @BeforeClass public static void setupClass() throws Exception { - topicAdminClient = - TopicAdminClient.create(TopicAdminSettings.newBuilder().setEndpoint(endPoint).build()); + topicAdminClient = TopicAdminClient.create(TopicAdminSettings.newBuilder().build()); subscriptionAdminClient = - SubscriptionAdminClient.create( - SubscriptionAdminSettings.newBuilder().setEndpoint(endPoint).build()); + SubscriptionAdminClient.create(SubscriptionAdminSettings.newBuilder().build()); projectId = ServiceOptions.getDefaultProjectId(); } @@ -216,7 +213,6 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) - .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -227,7 +223,7 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = Publisher.newBuilder(topicName).setEndpoint(endPoint).build(); + Publisher publisher = Publisher.newBuilder(topicName).build(); publisher .publish(PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8("msg1")).build()) .get(); @@ -437,7 +433,6 @@ public void receiveMessage( receiveQueue.offer(MessageAndConsumer.create(message, consumer)); } }) - .setEndpoint(endPoint) .build(); subscriber.addListener( new Subscriber.Listener() { @@ -448,8 +443,7 @@ public void failed(Subscriber.State from, Throwable failure) { MoreExecutors.directExecutor()); subscriber.startAsync(); - Publisher publisher = - Publisher.newBuilder(topicName).setEndpoint(endPoint).setEnableCompression(true).build(); + Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); String msg1 = generateMessage("msg1", 1000); String msg2 = generateMessage("msg2", 1500); diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 36eb03933..14becb78f 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -49,9 +49,13 @@ public static void publishWithCompressionExample(String projectId, String topicI Publisher publisher = null; try { - // Create a publisher instance bound to the topic with compression enabled and other default - // settings - publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build(); + // Create a publisher instance bound to the topic with compression enabled and a compression + // bytes threshold. + publisher = + Publisher.newBuilder(topicName) + .setEnableCompression(true) + .setCompressionBytesThreshold(500) + .build(); ByteString data = generateData("Hello!", 2000); PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); @@ -59,7 +63,7 @@ public static void publishWithCompressionExample(String projectId, String topicI // Once published, returns a server-assigned message id (unique within the topic) ApiFuture messageIdFuture = publisher.publish(pubsubMessage); String messageId = messageIdFuture.get(); - System.out.println("Published compressed message ID: " + messageId); + System.out.println("Published compressed message, ID:" + messageId); } finally { if (publisher != null) { // When finished with the publisher, shutdown to free up resources. @@ -69,7 +73,7 @@ public static void publishWithCompressionExample(String projectId, String topicI } } - // Generates data of given bytes by repeatedly concatenating a token. + /** Generates data of given bytes by repeatedly concatenating a token. */ // TODO(developer): Replace this method with your own data generation logic private static ByteString generateData(String token, int bytes) { String message = ""; @@ -80,6 +84,11 @@ private static ByteString generateData(String token, int bytes) { return ByteString.copyFromUtf8(message); } + /** + * Sets up logging to observe the outbound data (and its length) over the network to analyze the + * effectiveness of compression. A sample log line: + * [2022-03-02] FINE [id:..] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01... + */ private static void setUpLogs() throws IOException { String handlers = "handlers = java.util.logging.ConsoleHandler"; String handlerLevelProp = "java.util.logging.ConsoleHandler.level = ALL"; From 061f6795a9f9198cd8bf1ba57ab97fe48a6926da Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 2 Mar 2022 12:31:46 -0500 Subject: [PATCH 32/52] Addressing checkstyle --- .../src/main/java/pubsub/PublishWithCompressionExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 14becb78f..e625e335d 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -87,7 +87,7 @@ private static ByteString generateData(String token, int bytes) { /** * Sets up logging to observe the outbound data (and its length) over the network to analyze the * effectiveness of compression. A sample log line: - * [2022-03-02] FINE [id:..] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01... + * [2022-03-02] FINE [] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01.. */ private static void setUpLogs() throws IOException { String handlers = "handlers = java.util.logging.ConsoleHandler"; From 86a8327f7a8d9ce5d5431ca0200e728dbdcda70b Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Wed, 2 Mar 2022 14:56:50 -0500 Subject: [PATCH 33/52] Addressed PR comment --- .../src/main/java/com/google/cloud/pubsub/v1/Publisher.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 3be7fd019..f1a5fdb86 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -91,6 +91,8 @@ public class Publisher implements PublisherInterface { private static final Logger logger = Logger.getLogger(Publisher.class.getName()); + private static final String GZIP_COMPRESSION = "gzip"; + private final String topicName; private final BatchingSettings batchingSettings; @@ -445,7 +447,7 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { private ApiFuture publishCall(OutstandingBatch outstandingBatch) { if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { GrpcCallContext context = GrpcCallContext.createDefault(); - context = context.withCallOptions(CallOptions.DEFAULT.withCompression("gzip")); + context = context.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); return publisherStub .publishCallable() .futureCall( From 1abc9f83c471729b5f1ed814aad52fd1cd8291e6 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 10 Mar 2022 16:02:33 -0500 Subject: [PATCH 34/52] Addressing PR comment to put a Precondition for compression and its threshold --- .../com/google/cloud/pubsub/v1/Publisher.java | 15 +++++++---- .../cloud/pubsub/v1/PublisherImplTest.java | 25 ++++++++++++++++++- 2 files changed, 34 insertions(+), 6 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index f1a5fdb86..0b6fa62ae 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -119,6 +119,7 @@ public class Publisher implements PublisherInterface { private MessageFlowController flowController = null; private final boolean enableCompression; + private final boolean enableCompressionBytesThreshold; private final long compressionBytesThreshold; /** The maximum number of messages in one request. Defined by the API. */ @@ -148,6 +149,7 @@ private Publisher(Builder builder) throws IOException { this.enableMessageOrdering = builder.enableMessageOrdering; this.messageTransform = builder.messageTransform; this.enableCompression = builder.enableCompression; + this.enableCompressionBytesThreshold = builder.enableCompressionBytesThreshold; this.compressionBytesThreshold = builder.compressionBytesThreshold; messagesBatches = new HashMap<>(); @@ -212,11 +214,6 @@ public String getTopicNameString() { return topicName; } - /** Returns True/False if compression is enabled or disabled respectively. */ - public boolean getEnableCompression() { - return this.enableCompression; - } - /** * Schedules the publishing of a message. The publishing of the message may occur immediately or * be delayed based on the publisher batching options. @@ -256,6 +253,12 @@ public ApiFuture publish(PubsubMessage message) { + "Publisher client. Please create a Publisher client with " + "setEnableMessageOrdering(true) in the builder."); + Preconditions.checkState( + !enableCompressionBytesThreshold || enableCompression, + "Cannot publish a message with compression bytes threshold when compression is not enabled " + + "in the Publisher client. Please create a Publisher client with " + + "setEnableCompression(true) in the builder."); + final OutstandingPublish outstandingPublish = new OutstandingPublish(messageTransform.apply(message)); @@ -747,6 +750,7 @@ public PubsubMessage apply(PubsubMessage input) { }; private boolean enableCompression = DEFAULT_ENABLE_COMPRESSION; + private boolean enableCompressionBytesThreshold = DEFAULT_ENABLE_COMPRESSION; private long compressionBytesThreshold = DEFAULT_COMPRESSION_BYTES_THRESHOLD; private Builder(String topic) { @@ -871,6 +875,7 @@ public Builder setEnableCompression(boolean enableCompression) { * with this method. */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { + this.enableCompressionBytesThreshold = true; this.compressionBytesThreshold = compressionBytesThreshold; return this; } diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java index 1374ffad4..f82530bea 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java @@ -293,8 +293,8 @@ public void testPublishWithCompression() throws Exception { .setDelayThreshold(Duration.ofSeconds(100)) .build()) .setEnableCompression(true) + .setCompressionBytesThreshold(100) .build(); - assertTrue(publisher.getEnableCompression()); testPublisherServiceImpl.addPublishResponse( PublishResponse.newBuilder().addMessageIds("1").addMessageIds("2")); @@ -307,6 +307,29 @@ public void testPublishWithCompression() throws Exception { shutdownTestPublisher(publisher); } + @Test + public void testCompressionBytesThresholdWhenCompressionDisabled_throwsException() + throws Exception { + // Compression is not enabled but compression bytes theshold is set + Publisher publisher = + getTestPublisherBuilder() + .setBatchingSettings( + Publisher.Builder.DEFAULT_BATCHING_SETTINGS + .toBuilder() + .setElementCountThreshold(2L) + .setDelayThreshold(Duration.ofSeconds(100)) + .build()) + .setCompressionBytesThreshold(200) + .build(); + try { + ApiFuture publishFuture = sendTestMessage(publisher, "A"); + fail("Should have thrown an IllegalStateException"); + } catch (IllegalStateException expected) { + // expected + } + shutdownTestPublisher(publisher); + } + private ApiFuture sendTestMessage(Publisher publisher, String data) { return publisher.publish( PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(data)).build()); From 68484e5dfacdba56dba78d9da56a55a5a7d034a6 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 11 Mar 2022 15:33:54 -0500 Subject: [PATCH 35/52] Addressing PR review --- .../com/google/cloud/pubsub/v1/Publisher.java | 7 +++---- .../pubsub/PublishWithCompressionExample.java | 18 ++++-------------- 2 files changed, 7 insertions(+), 18 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 0b6fa62ae..2d2026e7a 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -863,16 +863,15 @@ public Builder setEndpoint(String endpoint) { return this; } - /** Gives the ability to enable gRPC compression. */ + /** Gives the ability to enable transport compression. */ public Builder setEnableCompression(boolean enableCompression) { this.enableCompression = enableCompression; return this; } /** - * Gives the ability to set the threshold in bytes above which gRPC compression happens. But to - * enable gRPC compression at the first place, setEnableCompression(true) should be called along - * with this method. + * Sets the threshold (in bytes) above which messages are compressed for transport. + * Only takes effect if setEnableCompression(true) is also called." */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { this.enableCompressionBytesThreshold = true; diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index e625e335d..71eb83679 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -1,5 +1,5 @@ /* - * Copyright 2020 Google LLC + * Copyright 2022 Google LLC * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -54,10 +54,11 @@ public static void publishWithCompressionExample(String projectId, String topicI publisher = Publisher.newBuilder(topicName) .setEnableCompression(true) - .setCompressionBytesThreshold(500) + .setCompressionBytesThreshold(10) .build(); - ByteString data = generateData("Hello!", 2000); + String message = "Hello World!"; + ByteString data = ByteString.copyFromUtf8(message); PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); // Once published, returns a server-assigned message id (unique within the topic) @@ -73,17 +74,6 @@ public static void publishWithCompressionExample(String projectId, String topicI } } - /** Generates data of given bytes by repeatedly concatenating a token. */ - // TODO(developer): Replace this method with your own data generation logic - private static ByteString generateData(String token, int bytes) { - String message = ""; - int tokenBytes = token.length(); - for (int i = 0; i < Math.floor(bytes / tokenBytes) + 1; i++) { - message = message.concat(token); - } - return ByteString.copyFromUtf8(message); - } - /** * Sets up logging to observe the outbound data (and its length) over the network to analyze the * effectiveness of compression. A sample log line: From fc2378e5b4d52750878b2fd291600365dc7505dd Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 11 Mar 2022 16:52:23 -0500 Subject: [PATCH 36/52] Removing logging from example --- .../pubsub/PublishWithCompressionExample.java | 32 ------------------- 1 file changed, 32 deletions(-) diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java index 71eb83679..7457c8c5d 100644 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java @@ -35,11 +35,7 @@ public static void main(String... args) throws Exception { // TODO(developer): Replace these variables before running the sample. String projectId = "your-project-id"; String topicId = "your-topic-id"; - boolean allowLogging = false; // Set to true to get the stdout logs - if (allowLogging) { - setUpLogs(); - } publishWithCompressionExample(projectId, topicId); } @@ -73,33 +69,5 @@ public static void publishWithCompressionExample(String projectId, String topicI } } } - - /** - * Sets up logging to observe the outbound data (and its length) over the network to analyze the - * effectiveness of compression. A sample log line: - * [2022-03-02] FINE [] OUTBOUND DATA: streamId=3 padding=0 endStream=true length=196 bytes=01.. - */ - private static void setUpLogs() throws IOException { - String handlers = "handlers = java.util.logging.ConsoleHandler"; - String handlerLevelProp = "java.util.logging.ConsoleHandler.level = ALL"; - String fineProp = ".level = FINE"; - String handlerFormatterProp = - "java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter"; - String format = "java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n"; - - LogManager.getLogManager() - .readConfiguration( - new ByteArrayInputStream( - (handlers - + "\n" - + handlerLevelProp - + "\n" - + fineProp - + "\n" - + handlerFormatterProp - + "\n" - + format) - .getBytes(StandardCharsets.UTF_8))); - } } // [END pubsub_publish_with_compression] From 7c34ef60b738309bfe513b2b7cd4a30971a8a5dc Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 11 Mar 2022 16:53:22 -0500 Subject: [PATCH 37/52] Adding logging properties --- samples/snippets/src/main/resources/logging.properties | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 samples/snippets/src/main/resources/logging.properties diff --git a/samples/snippets/src/main/resources/logging.properties b/samples/snippets/src/main/resources/logging.properties new file mode 100644 index 000000000..1f482dad2 --- /dev/null +++ b/samples/snippets/src/main/resources/logging.properties @@ -0,0 +1,5 @@ +handlers = java.util.logging.ConsoleHandler +java.util.logging.ConsoleHandler.level = ALL +.level = FINE +java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter +java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n \ No newline at end of file From 92c2824c851387bf1e23ed949a0999d736a34353 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Tue, 15 Mar 2022 14:04:35 -0400 Subject: [PATCH 38/52] Making the publish call unified with context as per PR comments --- .../com/google/cloud/pubsub/v1/Publisher.java | 30 +++++++------------ 1 file changed, 11 insertions(+), 19 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 2d2026e7a..f78edda45 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -448,26 +448,18 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { + GrpcCallContext context = GrpcCallContext.createDefault(); if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { - GrpcCallContext context = GrpcCallContext.createDefault(); context = context.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); - return publisherStub - .publishCallable() - .futureCall( - PublishRequest.newBuilder() - .setTopic(topicName) - .addAllMessages(outstandingBatch.getMessages()) - .build(), - context); - } else { - return publisherStub - .publishCallable() - .futureCall( - PublishRequest.newBuilder() - .setTopic(topicName) - .addAllMessages(outstandingBatch.getMessages()) - .build()); } + return publisherStub + .publishCallable() + .futureCall( + PublishRequest.newBuilder() + .setTopic(topicName) + .addAllMessages(outstandingBatch.getMessages()) + .build(), + context); } private void publishOutstandingBatch(final OutstandingBatch outstandingBatch) { @@ -870,8 +862,8 @@ public Builder setEnableCompression(boolean enableCompression) { } /** - * Sets the threshold (in bytes) above which messages are compressed for transport. - * Only takes effect if setEnableCompression(true) is also called." + * Sets the threshold (in bytes) above which messages are compressed for transport. Only takes + * effect if setEnableCompression(true) is also called." */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { this.enableCompressionBytesThreshold = true; From 265350ad5d3cdc82a969fadbe2f33e83f892d32d Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 21 Mar 2022 16:17:41 -0400 Subject: [PATCH 39/52] Removing sample code as per tianzi@'s comments --- README.md | 1 - .../pubsub/PublishWithCompressionExample.java | 73 ------------------- .../src/main/resources/logging.properties | 5 -- 3 files changed, 79 deletions(-) delete mode 100644 samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java delete mode 100644 samples/snippets/src/main/resources/logging.properties diff --git a/README.md b/README.md index 4575723d9..593f91dec 100644 --- a/README.md +++ b/README.md @@ -272,7 +272,6 @@ Samples are in the [`samples/`](https://github.com/googleapis/java-pubsub/tree/m | Publish With Flow Control Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithFlowControlExample.java) | | Publish With Ordering Keys | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithOrderingKeys.java) | | Publish With Retry Settings Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithRetrySettingsExample.java) | -| Publish With Compression Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java) | | Publisher Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/PublisherExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/PublisherExample.java) | | Receive Messages With Delivery Attempts Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/ReceiveMessagesWithDeliveryAttemptsExample.java) | | Remove Dead Letter Policy Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | [![Open in Cloud Shell][shell_img]](https://console.cloud.google.com/cloudshell/open?git_repo=https://github.com/googleapis/java-pubsub&page=editor&open_in_editor=samples/snippets/src/main/java/pubsub/RemoveDeadLetterPolicyExample.java) | diff --git a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java b/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java deleted file mode 100644 index 7457c8c5d..000000000 --- a/samples/snippets/src/main/java/pubsub/PublishWithCompressionExample.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright 2022 Google LLC - * - * Licensed 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 pubsub; - -// [START pubsub_publish_with_compression] -import com.google.api.core.ApiFuture; -import com.google.cloud.pubsub.v1.Publisher; -import com.google.protobuf.ByteString; -import com.google.pubsub.v1.PubsubMessage; -import com.google.pubsub.v1.TopicName; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.logging.LogManager; - -public class PublishWithCompressionExample { - - public static void main(String... args) throws Exception { - // TODO(developer): Replace these variables before running the sample. - String projectId = "your-project-id"; - String topicId = "your-topic-id"; - - publishWithCompressionExample(projectId, topicId); - } - - public static void publishWithCompressionExample(String projectId, String topicId) - throws IOException, ExecutionException, InterruptedException { - TopicName topicName = TopicName.of(projectId, topicId); - - Publisher publisher = null; - try { - // Create a publisher instance bound to the topic with compression enabled and a compression - // bytes threshold. - publisher = - Publisher.newBuilder(topicName) - .setEnableCompression(true) - .setCompressionBytesThreshold(10) - .build(); - - String message = "Hello World!"; - ByteString data = ByteString.copyFromUtf8(message); - PubsubMessage pubsubMessage = PubsubMessage.newBuilder().setData(data).build(); - - // Once published, returns a server-assigned message id (unique within the topic) - ApiFuture messageIdFuture = publisher.publish(pubsubMessage); - String messageId = messageIdFuture.get(); - System.out.println("Published compressed message, ID:" + messageId); - } finally { - if (publisher != null) { - // When finished with the publisher, shutdown to free up resources. - publisher.shutdown(); - publisher.awaitTermination(1, TimeUnit.MINUTES); - } - } - } -} -// [END pubsub_publish_with_compression] diff --git a/samples/snippets/src/main/resources/logging.properties b/samples/snippets/src/main/resources/logging.properties deleted file mode 100644 index 1f482dad2..000000000 --- a/samples/snippets/src/main/resources/logging.properties +++ /dev/null @@ -1,5 +0,0 @@ -handlers = java.util.logging.ConsoleHandler -java.util.logging.ConsoleHandler.level = ALL -.level = FINE -java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter -java.util.logging.SimpleFormatter.format=[%1$tF %1$tT] %4$-5s %5$s %n \ No newline at end of file From 2e9a7389aaf0639846ccd146a7ccdc8a903c744d Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Mon, 11 Apr 2022 11:45:37 -0400 Subject: [PATCH 40/52] Minor fixes --- .../com/google/cloud/pubsub/v1/Publisher.java | 2 +- .../google/cloud/pubsub/it/ITPubSubTest.java | 20 ++++++------------- .../src/test/java/pubsub/PublisherIT.java | 5 ----- 3 files changed, 7 insertions(+), 20 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index f78edda45..d290b978e 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -711,7 +711,7 @@ public static final class Builder { .setExecutorThreadCount(THREADS_PER_CPU * Runtime.getRuntime().availableProcessors()) .build(); static final boolean DEFAULT_ENABLE_COMPRESSION = false; - static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 500L; + static final long DEFAULT_COMPRESSION_BYTES_THRESHOLD = 240L; String topicName; private String endpoint = PublisherStubSettings.getDefaultEndpoint(); diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index c5e512e65..8d61ab6c5 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -24,14 +24,7 @@ import com.google.api.gax.rpc.PermissionDeniedException; import com.google.auto.value.AutoValue; import com.google.cloud.ServiceOptions; -import com.google.cloud.pubsub.v1.AckReplyConsumer; -import com.google.cloud.pubsub.v1.MessageReceiver; -import com.google.cloud.pubsub.v1.Publisher; -import com.google.cloud.pubsub.v1.Subscriber; -import com.google.cloud.pubsub.v1.SubscriptionAdminClient; -import com.google.cloud.pubsub.v1.SubscriptionAdminSettings; -import com.google.cloud.pubsub.v1.TopicAdminClient; -import com.google.cloud.pubsub.v1.TopicAdminSettings; +import com.google.cloud.pubsub.v1.*; import com.google.common.util.concurrent.MoreExecutors; import com.google.iam.v1.Binding; import com.google.iam.v1.GetIamPolicyRequest; @@ -90,9 +83,8 @@ static MessageAndConsumerWithResponse create( @BeforeClass public static void setupClass() throws Exception { - topicAdminClient = TopicAdminClient.create(TopicAdminSettings.newBuilder().build()); - subscriptionAdminClient = - SubscriptionAdminClient.create(SubscriptionAdminSettings.newBuilder().build()); + topicAdminClient = TopicAdminClient.create(); + subscriptionAdminClient = SubscriptionAdminClient.create(); projectId = ServiceOptions.getDefaultProjectId(); } @@ -420,7 +412,7 @@ public void testPublishSubscribeWithCompression() throws Exception { topicAdminClient.createTopic(topicName); subscriptionAdminClient.createSubscription( - getSubscription(subscriptionName, topicName, PushConfig.newBuilder().build(), 10)); + getSubscription(subscriptionName, topicName, PushConfig.newBuilder().build(), 10, false)); final BlockingQueue receiveQueue = new LinkedBlockingQueue<>(); Subscriber subscriber = @@ -457,12 +449,12 @@ public void failed(Subscriber.State from, Throwable failure) { publisher.awaitTermination(1, TimeUnit.MINUTES); // Ack the first message. - MessageAndConsumer toAck1 = pollQueue(receiveQueue); + MessageAndConsumer toAck1 = pollQueueMessageAndConsumer(receiveQueue); assertThat(toAck1.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg1)); toAck1.consumer().ack(); // Ack the second message. - MessageAndConsumer toAck2 = pollQueue(receiveQueue); + MessageAndConsumer toAck2 = pollQueueMessageAndConsumer(receiveQueue); assertThat(toAck2.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg2)); toAck2.consumer().ack(); diff --git a/samples/snippets/src/test/java/pubsub/PublisherIT.java b/samples/snippets/src/test/java/pubsub/PublisherIT.java index 48525e7d6..b22912d47 100644 --- a/samples/snippets/src/test/java/pubsub/PublisherIT.java +++ b/samples/snippets/src/test/java/pubsub/PublisherIT.java @@ -124,10 +124,5 @@ public void testPublisher() throws Exception { for (int i = 1; i <= 4; i++) { assertThat(bout.toString()).contains("message" + i); } - - bout.reset(); - // Test publish with compression. - PublishWithCompressionExample.publishWithCompressionExample(projectId, topicId); - assertThat(bout.toString()).contains("Published compressed message ID: "); } } From cef13ea88a517c36bf13b30663765b9629269a8e Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 28 Apr 2022 12:13:52 -0400 Subject: [PATCH 41/52] Fixing IT --- .../test/java/com/google/cloud/pubsub/it/ITPubSubTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java index 8d61ab6c5..290b9927d 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java @@ -450,14 +450,14 @@ public void failed(Subscriber.State from, Throwable failure) { // Ack the first message. MessageAndConsumer toAck1 = pollQueueMessageAndConsumer(receiveQueue); - assertThat(toAck1.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg1)); toAck1.consumer().ack(); // Ack the second message. MessageAndConsumer toAck2 = pollQueueMessageAndConsumer(receiveQueue); - assertThat(toAck2.message().getData()).isEqualTo(ByteString.copyFromUtf8(msg2)); toAck2.consumer().ack(); + assertNotEquals(toAck1.message().getData(), toAck2.message().getData()); + subscriber.stopAsync().awaitTerminated(); subscriptionAdminClient.deleteSubscription(subscriptionName); topicAdminClient.deleteTopic(topicName); From ed736958ac0daa46ba4add8c2e9d48709e87c6aa Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 5 May 2022 14:21:19 -0400 Subject: [PATCH 42/52] Creating a class variable publishContext to remove the overhead of GrpcCallContext.createDefault() with every publish call --- .../main/java/com/google/cloud/pubsub/v1/Publisher.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index d290b978e..35cb848ff 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -122,6 +122,8 @@ public class Publisher implements PublisherInterface { private final boolean enableCompressionBytesThreshold; private final long compressionBytesThreshold; + private GrpcCallContext publishContext; + /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { return 1000L; @@ -202,6 +204,7 @@ private Publisher(Builder builder) throws IOException { backgroundResources = new BackgroundResourceAggregation(backgroundResourceList); shutdown = new AtomicBoolean(false); messagesWaiter = new Waiter(); + this.publishContext = GrpcCallContext.createDefault(); } /** Topic which the publisher publishes to. */ @@ -448,9 +451,8 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { - GrpcCallContext context = GrpcCallContext.createDefault(); if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { - context = context.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); + publishContext = publishContext.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); } return publisherStub .publishCallable() @@ -459,7 +461,7 @@ private ApiFuture publishCall(OutstandingBatch outstandingBatch .setTopic(topicName) .addAllMessages(outstandingBatch.getMessages()) .build(), - context); + publishContext); } private void publishOutstandingBatch(final OutstandingBatch outstandingBatch) { From db27b123a3618f699625f17b8f630426488ca175 Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Thu, 5 May 2022 14:25:06 -0400 Subject: [PATCH 43/52] fixing lint format --- .../src/main/java/com/google/cloud/pubsub/v1/Publisher.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 35cb848ff..13c82a207 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -452,7 +452,8 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { private ApiFuture publishCall(OutstandingBatch outstandingBatch) { if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { - publishContext = publishContext.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); + publishContext = + publishContext.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); } return publisherStub .publishCallable() From 4d44679a109b7262b26b6b39c157e9c414fd70be Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 6 May 2022 15:49:35 -0400 Subject: [PATCH 44/52] Addressed PR comments --- .../com/google/cloud/pubsub/v1/Publisher.java | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java index 13c82a207..399d99658 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java @@ -119,10 +119,10 @@ public class Publisher implements PublisherInterface { private MessageFlowController flowController = null; private final boolean enableCompression; - private final boolean enableCompressionBytesThreshold; private final long compressionBytesThreshold; - private GrpcCallContext publishContext; + private final GrpcCallContext publishContext; + private final GrpcCallContext publishContextWithCompression; /** The maximum number of messages in one request. Defined by the API. */ public static long getApiMaxRequestElementCount() { @@ -151,7 +151,6 @@ private Publisher(Builder builder) throws IOException { this.enableMessageOrdering = builder.enableMessageOrdering; this.messageTransform = builder.messageTransform; this.enableCompression = builder.enableCompression; - this.enableCompressionBytesThreshold = builder.enableCompressionBytesThreshold; this.compressionBytesThreshold = builder.compressionBytesThreshold; messagesBatches = new HashMap<>(); @@ -205,6 +204,9 @@ private Publisher(Builder builder) throws IOException { shutdown = new AtomicBoolean(false); messagesWaiter = new Waiter(); this.publishContext = GrpcCallContext.createDefault(); + this.publishContextWithCompression = + GrpcCallContext.createDefault() + .withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); } /** Topic which the publisher publishes to. */ @@ -256,12 +258,6 @@ public ApiFuture publish(PubsubMessage message) { + "Publisher client. Please create a Publisher client with " + "setEnableMessageOrdering(true) in the builder."); - Preconditions.checkState( - !enableCompressionBytesThreshold || enableCompression, - "Cannot publish a message with compression bytes threshold when compression is not enabled " - + "in the Publisher client. Please create a Publisher client with " - + "setEnableCompression(true) in the builder."); - final OutstandingPublish outstandingPublish = new OutstandingPublish(messageTransform.apply(message)); @@ -451,9 +447,9 @@ private void publishAllWithoutInflightForKey(final String orderingKey) { } private ApiFuture publishCall(OutstandingBatch outstandingBatch) { + GrpcCallContext context = publishContext; if (enableCompression && outstandingBatch.batchSizeBytes >= compressionBytesThreshold) { - publishContext = - publishContext.withCallOptions(CallOptions.DEFAULT.withCompression(GZIP_COMPRESSION)); + context = publishContextWithCompression; } return publisherStub .publishCallable() @@ -462,7 +458,7 @@ private ApiFuture publishCall(OutstandingBatch outstandingBatch .setTopic(topicName) .addAllMessages(outstandingBatch.getMessages()) .build(), - publishContext); + context); } private void publishOutstandingBatch(final OutstandingBatch outstandingBatch) { @@ -745,7 +741,6 @@ public PubsubMessage apply(PubsubMessage input) { }; private boolean enableCompression = DEFAULT_ENABLE_COMPRESSION; - private boolean enableCompressionBytesThreshold = DEFAULT_ENABLE_COMPRESSION; private long compressionBytesThreshold = DEFAULT_COMPRESSION_BYTES_THRESHOLD; private Builder(String topic) { @@ -869,7 +864,6 @@ public Builder setEnableCompression(boolean enableCompression) { * effect if setEnableCompression(true) is also called." */ public Builder setCompressionBytesThreshold(long compressionBytesThreshold) { - this.enableCompressionBytesThreshold = true; this.compressionBytesThreshold = compressionBytesThreshold; return this; } From b1a2d370496a0c1a3eb3e57d3fbb6be8b384445a Mon Sep 17 00:00:00 2001 From: Rajanya Dhar Date: Fri, 6 May 2022 18:16:15 -0400 Subject: [PATCH 45/52] Removing test --- .../cloud/pubsub/v1/PublisherImplTest.java | 23 ------------------- 1 file changed, 23 deletions(-) diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java index f82530bea..9985efc6b 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java @@ -307,29 +307,6 @@ public void testPublishWithCompression() throws Exception { shutdownTestPublisher(publisher); } - @Test - public void testCompressionBytesThresholdWhenCompressionDisabled_throwsException() - throws Exception { - // Compression is not enabled but compression bytes theshold is set - Publisher publisher = - getTestPublisherBuilder() - .setBatchingSettings( - Publisher.Builder.DEFAULT_BATCHING_SETTINGS - .toBuilder() - .setElementCountThreshold(2L) - .setDelayThreshold(Duration.ofSeconds(100)) - .build()) - .setCompressionBytesThreshold(200) - .build(); - try { - ApiFuture publishFuture = sendTestMessage(publisher, "A"); - fail("Should have thrown an IllegalStateException"); - } catch (IllegalStateException expected) { - // expected - } - shutdownTestPublisher(publisher); - } - private ApiFuture sendTestMessage(Publisher publisher, String data) { return publisher.publish( PubsubMessage.newBuilder().setData(ByteString.copyFromUtf8(data)).build()); From a26562ed43d233eaa88705cb073ed6fe601cb57f Mon Sep 17 00:00:00 2001 From: WhiteSource Renovate Date: Fri, 29 Apr 2022 22:24:28 +0200 Subject: [PATCH 46/52] build(deps): update dependency com.google.cloud:google-cloud-shared-config to v1.4.0 (#1105) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit [![WhiteSource Renovate](https://app.renovatebot.com/images/banner.svg)](https://renovatebot.com) This PR contains the following updates: | Package | Change | Age | Adoption | Passing | Confidence | |---|---|---|---|---|---| | [com.google.cloud:google-cloud-shared-config](https://togithub.com/googleapis/java-shared-config) | `1.3.3` -> `1.4.0` | [![age](https://badges.renovateapi.com/packages/maven/com.google.cloud:google-cloud-shared-config/1.4.0/age-slim)](https://docs.renovatebot.com/merge-confidence/) | [![adoption](https://badges.renovateapi.com/packages/maven/com.google.cloud:google-cloud-shared-config/1.4.0/adoption-slim)](https://docs.renovatebot.com/merge-confidence/) | [![passing](https://badges.renovateapi.com/packages/maven/com.google.cloud:google-cloud-shared-config/1.4.0/compatibility-slim/1.3.3)](https://docs.renovatebot.com/merge-confidence/) | [![confidence](https://badges.renovateapi.com/packages/maven/com.google.cloud:google-cloud-shared-config/1.4.0/confidence-slim/1.3.3)](https://docs.renovatebot.com/merge-confidence/) | --- ### Release Notes
googleapis/java-shared-config ### [`v1.4.0`](https://togithub.com/googleapis/java-shared-config/blob/HEAD/CHANGELOG.md#​140-httpsgithubcomgoogleapisjava-shared-configcomparev133v140-2022-04-28) [Compare Source](https://togithub.com/googleapis/java-shared-config/compare/v1.3.3...v1.4.0) ##### Features - **java:** remove native image module ([#​471](https://togithub.com/googleapis/java-shared-config/issues/471)) ([7fcba01](https://togithub.com/googleapis/java-shared-config/commit/7fcba016b3138d7beaa4e962853f9bc80f59438c)) ##### [1.3.3](https://togithub.com/googleapis/java-shared-config/compare/v1.3.2...v1.3.3) (2022-04-19) ##### Bug Fixes - **java:** remove protobuf feature from native profile ([#​461](https://togithub.com/googleapis/java-shared-config/issues/461)) ([ffd07cb](https://togithub.com/googleapis/java-shared-config/commit/ffd07cb18ee7d45d4daee1d9ea6f6d321fdca874)) ##### Dependencies - update dependency com.google.cloud:native-image-support to v0.12.11 ([#​459](https://togithub.com/googleapis/java-shared-config/issues/459)) ([d20008d](https://togithub.com/googleapis/java-shared-config/commit/d20008df15209708fdf9d06828b567778190f4d0)) - update dependency com.google.cloud:native-image-support to v0.13.1 ([#​465](https://togithub.com/googleapis/java-shared-config/issues/465)) ([b202064](https://togithub.com/googleapis/java-shared-config/commit/b2020648816feb4740ad70acedfed470d7da5bcf)) ##### [1.3.2](https://togithub.com/googleapis/java-shared-config/compare/v1.3.1...v1.3.2) (2022-03-28) ##### Dependencies - revert google-java-format to 1.7 ([#​453](https://togithub.com/googleapis/java-shared-config/issues/453)) ([cbc777f](https://togithub.com/googleapis/java-shared-config/commit/cbc777f3e9ab75edb6fa2e0268a7446ae4111725)) ##### [1.3.1](https://togithub.com/googleapis/java-shared-config/compare/v1.3.0...v1.3.1) (2022-03-25) ##### Dependencies - update dependency com.google.cloud:native-image-support to v0.12.10 ([#​443](https://togithub.com/googleapis/java-shared-config/issues/443)) ([5b39d5e](https://togithub.com/googleapis/java-shared-config/commit/5b39d5ee15121f052226ff873b6ab101e9c71de5)) - update dependency com.google.googlejavaformat:google-java-format to v1.15.0 ([#​426](https://togithub.com/googleapis/java-shared-config/issues/426)) ([4c3c4b6](https://togithub.com/googleapis/java-shared-config/commit/4c3c4b66129632181e6bc363a0ecccf4f5aac914)) - update dependency org.graalvm.buildtools:junit-platform-native to v0.9.11 ([#​448](https://togithub.com/googleapis/java-shared-config/issues/448)) ([f7f518e](https://togithub.com/googleapis/java-shared-config/commit/f7f518e87d1d9feb9ac54d7c099f97d8751ee3da)) - update dependency org.graalvm.buildtools:native-maven-plugin to v0.9.11 ([#​449](https://togithub.com/googleapis/java-shared-config/issues/449)) ([3e1c0b5](https://togithub.com/googleapis/java-shared-config/commit/3e1c0b5a1d2f4a0db88c06a0d683ed90cbc745e2))
--- ### Configuration 📅 **Schedule**: At any time (no schedule defined). 🚦 **Automerge**: Disabled by config. Please merge this manually once you are satisfied. ♻ **Rebasing**: Whenever PR becomes conflicted, or you tick the rebase/retry checkbox. 🔕 **Ignore**: Close this PR and you won't be reminded about this update again. --- - [ ] If you want to rebase/retry this PR, click this checkbox. --- This PR has been generated by [WhiteSource Renovate](https://renovate.whitesourcesoftware.com). View repository job log [here](https://app.renovatebot.com/dashboard#github/googleapis/java-pubsub). --- google-cloud-pubsub-bom/pom.xml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/google-cloud-pubsub-bom/pom.xml b/google-cloud-pubsub-bom/pom.xml index 5412ea9ed..3fb58983b 100644 --- a/google-cloud-pubsub-bom/pom.xml +++ b/google-cloud-pubsub-bom/pom.xml @@ -8,7 +8,7 @@ com.google.cloud google-cloud-shared-config - 1.3.3 + 1.4.0 Google Cloud pubsub BOM diff --git a/pom.xml b/pom.xml index f68a093af..716e6366e 100644 --- a/pom.xml +++ b/pom.xml @@ -14,7 +14,7 @@ com.google.cloud google-cloud-shared-config - 1.3.3 + 1.4.0 From d9130483c6a3492e6f054382c039d020728103be Mon Sep 17 00:00:00 2001 From: Mike Micatka <31972785+mmicatka@users.noreply.github.com> Date: Mon, 2 May 2022 15:04:29 -0400 Subject: [PATCH 47/52] fix: added exactly once delivery files to owlbot config (#1106) --- .github/.OwlBot.yaml | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/.github/.OwlBot.yaml b/.github/.OwlBot.yaml index 3fd3247a0..e768d51d1 100644 --- a/.github/.OwlBot.yaml +++ b/.github/.OwlBot.yaml @@ -28,9 +28,8 @@ deep-preserve-regex: - "/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ProjectTopicName.java" - "/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicNames.java" - "/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UntypedTopicName.java" -- "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminSmokeTest.java" -- "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/WaiterTest.java" - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java" +- "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/CustomArgumentMatchers.java" - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/FakeClock.java" - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/FakePublisherServiceImpl.java" - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/FakeScheduledExecutorService.java" @@ -40,20 +39,28 @@ deep-preserve-regex: - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/PublisherImplTest.java" - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SequentialExecutorServiceTest.java" - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/StatusUtilTest.java" +- "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/StreamingSubscriberConnectionTest.java" - "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriberTest.java" -- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Waiter.java" -- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SequentialExecutorService.java" -- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/StatusUtil.java" -- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/StreamingSubscriberConnection.java" -- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Subscriber.java" -- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriberInterface.java" -- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriberStats.java" +- "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminSmokeTest.java" +- "/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/WaiterTest.java" - "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/AckReplyConsumer.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/AckReplyConsumerWithResponse.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/AckRequestData.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/AckResponse.java" - "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/MessageDispatcher.java" - "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/MessageReceiver.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/MessageReceiverWithAckResponse.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/ModackRequestData.java" - "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/OpenCensusUtil.java" - "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Publisher.java" - "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/PublisherInterface.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SequentialExecutorService.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/StatusUtil.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/StreamingSubscriberConnection.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Subscriber.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriberInterface.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriberStats.java" +- "/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/Waiter.java" deep-copy-regex: - source: "/google/pubsub/(v.*)/.*-java/proto-google-.*/src" From 5c9778abb70330659f18a1cc4bd1e9bfa44650d9 Mon Sep 17 00:00:00 2001 From: "gcf-owl-bot[bot]" <78513119+gcf-owl-bot[bot]@users.noreply.github.com> Date: Mon, 2 May 2022 21:54:11 +0000 Subject: [PATCH 48/52] chore(bazel): update version of Protobuf to v3.20.1 (#1079) - [ ] Regenerate this pull request now. PiperOrigin-RevId: 444328399 Source-Link: https://github.com/googleapis/googleapis/commit/c7ca416c2856aad6a3f7092924e56b8cf0cb2534 Source-Link: https://github.com/googleapis/googleapis-gen/commit/d61705453a62b3ecda78aa30c192840ebc5a8a90 Copy-Tag: eyJwIjoiLmdpdGh1Yi8uT3dsQm90LnlhbWwiLCJoIjoiZDYxNzA1NDUzYTYyYjNlY2RhNzhhYTMwYzE5Mjg0MGViYzVhOGE5MCJ9 feat: AuditConfig for IAM v1 PiperOrigin-RevId: 439356405 Source-Link: https://github.com/googleapis/googleapis/commit/afa2ba156bd5c83ad8168030ab801a8ca84ac819 Source-Link: https://github.com/googleapis/googleapis-gen/commit/3e40c17e1510c95fab58fc2143ccb61cceca5989 Copy-Tag: eyJwIjoiLmdpdGh1Yi8uT3dsQm90LnlhbWwiLCJoIjoiM2U0MGMxN2UxNTEwYzk1ZmFiNThmYzIxNDNjY2I2MWNjZWNhNTk4OSJ9 --- .../cloud/pubsub/v1/SchemaServiceClient.java | 2 ++ .../pubsub/v1/SubscriptionAdminClient.java | 2 ++ .../cloud/pubsub/v1/TopicAdminClient.java | 2 ++ .../pubsub/v1/SchemaServiceClientTest.java | 7 +++++ .../v1/SubscriptionAdminClientTest.java | 6 +++++ .../cloud/pubsub/v1/TopicAdminClientTest.java | 6 +++++ .../google/pubsub/v1/AcknowledgeRequest.java | 2 ++ .../google/pubsub/v1/CreateSchemaRequest.java | 2 ++ .../pubsub/v1/CreateSnapshotRequest.java | 21 ++++++++------- .../v1/CreateSnapshotRequestOrBuilder.java | 7 ++++- .../google/pubsub/v1/DeadLetterPolicy.java | 2 ++ .../google/pubsub/v1/DeleteSchemaRequest.java | 2 ++ .../pubsub/v1/DeleteSnapshotRequest.java | 2 ++ .../pubsub/v1/DeleteSubscriptionRequest.java | 2 ++ .../google/pubsub/v1/DeleteTopicRequest.java | 2 ++ .../pubsub/v1/DetachSubscriptionRequest.java | 2 ++ .../pubsub/v1/DetachSubscriptionResponse.java | 2 ++ .../google/pubsub/v1/ExpirationPolicy.java | 2 ++ .../google/pubsub/v1/GetSchemaRequest.java | 2 ++ .../google/pubsub/v1/GetSnapshotRequest.java | 2 ++ .../pubsub/v1/GetSubscriptionRequest.java | 2 ++ .../com/google/pubsub/v1/GetTopicRequest.java | 2 ++ .../google/pubsub/v1/ListSchemasRequest.java | 2 ++ .../google/pubsub/v1/ListSchemasResponse.java | 2 ++ .../pubsub/v1/ListSnapshotsRequest.java | 2 ++ .../pubsub/v1/ListSnapshotsResponse.java | 2 ++ .../pubsub/v1/ListSubscriptionsRequest.java | 2 ++ .../pubsub/v1/ListSubscriptionsResponse.java | 2 ++ .../pubsub/v1/ListTopicSnapshotsRequest.java | 2 ++ .../pubsub/v1/ListTopicSnapshotsResponse.java | 2 ++ .../v1/ListTopicSubscriptionsRequest.java | 2 ++ .../v1/ListTopicSubscriptionsResponse.java | 2 ++ .../google/pubsub/v1/ListTopicsRequest.java | 2 ++ .../google/pubsub/v1/ListTopicsResponse.java | 2 ++ .../pubsub/v1/MessageStoragePolicy.java | 2 ++ .../pubsub/v1/ModifyAckDeadlineRequest.java | 2 ++ .../pubsub/v1/ModifyPushConfigRequest.java | 2 ++ .../com/google/pubsub/v1/PublishRequest.java | 2 ++ .../com/google/pubsub/v1/PublishResponse.java | 2 ++ .../com/google/pubsub/v1/PubsubMessage.java | 21 ++++++++------- .../pubsub/v1/PubsubMessageOrBuilder.java | 7 ++++- .../com/google/pubsub/v1/PullRequest.java | 10 +++++++ .../pubsub/v1/PullRequestOrBuilder.java | 2 ++ .../com/google/pubsub/v1/PullResponse.java | 2 ++ .../java/com/google/pubsub/v1/PushConfig.java | 26 ++++++++++++------- .../google/pubsub/v1/PushConfigOrBuilder.java | 7 ++++- .../com/google/pubsub/v1/ReceivedMessage.java | 2 ++ .../com/google/pubsub/v1/RetryPolicy.java | 2 ++ .../java/com/google/pubsub/v1/Schema.java | 2 ++ .../com/google/pubsub/v1/SchemaSettings.java | 2 ++ .../com/google/pubsub/v1/SeekRequest.java | 5 +++- .../com/google/pubsub/v1/SeekResponse.java | 2 ++ .../java/com/google/pubsub/v1/Snapshot.java | 21 ++++++++------- .../google/pubsub/v1/SnapshotOrBuilder.java | 7 ++++- .../pubsub/v1/StreamingPullRequest.java | 2 ++ .../pubsub/v1/StreamingPullResponse.java | 8 ++++++ .../com/google/pubsub/v1/Subscription.java | 21 ++++++++------- .../pubsub/v1/SubscriptionOrBuilder.java | 7 ++++- .../main/java/com/google/pubsub/v1/Topic.java | 21 ++++++++------- .../com/google/pubsub/v1/TopicOrBuilder.java | 7 ++++- .../pubsub/v1/UpdateSnapshotRequest.java | 2 ++ .../pubsub/v1/UpdateSubscriptionRequest.java | 2 ++ .../google/pubsub/v1/UpdateTopicRequest.java | 2 ++ .../pubsub/v1/ValidateMessageRequest.java | 5 +++- .../pubsub/v1/ValidateMessageResponse.java | 2 ++ .../pubsub/v1/ValidateSchemaRequest.java | 2 ++ .../pubsub/v1/ValidateSchemaResponse.java | 2 ++ 67 files changed, 253 insertions(+), 63 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java index 562debea0..fd0f238e2 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java @@ -777,6 +777,7 @@ public final ValidateMessageResponse validateMessage(ValidateMessageRequest requ * SetIamPolicyRequest.newBuilder() * .setResource(ProjectName.of("[PROJECT]").toString()) * .setPolicy(Policy.newBuilder().build()) + * .setUpdateMask(FieldMask.newBuilder().build()) * .build(); * Policy response = schemaServiceClient.setIamPolicy(request); * } @@ -803,6 +804,7 @@ public final Policy setIamPolicy(SetIamPolicyRequest request) { * SetIamPolicyRequest.newBuilder() * .setResource(ProjectName.of("[PROJECT]").toString()) * .setPolicy(Policy.newBuilder().build()) + * .setUpdateMask(FieldMask.newBuilder().build()) * .build(); * ApiFuture future = schemaServiceClient.setIamPolicyCallable().futureCall(request); * // Do something. diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java index f5dfe6990..8f11330eb 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java @@ -2984,6 +2984,7 @@ public final UnaryCallable seekCallable() { * SetIamPolicyRequest.newBuilder() * .setResource(ProjectName.of("[PROJECT]").toString()) * .setPolicy(Policy.newBuilder().build()) + * .setUpdateMask(FieldMask.newBuilder().build()) * .build(); * Policy response = subscriptionAdminClient.setIamPolicy(request); * } @@ -3041,6 +3042,7 @@ public final Policy setIamPolicy(String resource, Policy policy) { * SetIamPolicyRequest.newBuilder() * .setResource(ProjectName.of("[PROJECT]").toString()) * .setPolicy(Policy.newBuilder().build()) + * .setUpdateMask(FieldMask.newBuilder().build()) * .build(); * ApiFuture future = subscriptionAdminClient.setIamPolicyCallable().futureCall(request); * // Do something. diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java index 61b154c1e..1d3da0057 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java @@ -1233,6 +1233,7 @@ public final DetachSubscriptionResponse detachSubscription(DetachSubscriptionReq * SetIamPolicyRequest.newBuilder() * .setResource(ProjectName.of("[PROJECT]").toString()) * .setPolicy(Policy.newBuilder().build()) + * .setUpdateMask(FieldMask.newBuilder().build()) * .build(); * Policy response = topicAdminClient.setIamPolicy(request); * } @@ -1290,6 +1291,7 @@ public final Policy setIamPolicy(String resource, Policy policy) { * SetIamPolicyRequest.newBuilder() * .setResource(ProjectName.of("[PROJECT]").toString()) * .setPolicy(Policy.newBuilder().build()) + * .setUpdateMask(FieldMask.newBuilder().build()) * .build(); * ApiFuture future = topicAdminClient.setIamPolicyCallable().futureCall(request); * // Do something. diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java index ace5e7d42..864f9f013 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java @@ -26,6 +26,7 @@ import com.google.api.gax.rpc.ApiClientHeaderProvider; import com.google.api.gax.rpc.InvalidArgumentException; import com.google.common.collect.Lists; +import com.google.iam.v1.AuditConfig; import com.google.iam.v1.Binding; import com.google.iam.v1.GetIamPolicyRequest; import com.google.iam.v1.GetPolicyOptions; @@ -36,6 +37,7 @@ import com.google.protobuf.AbstractMessage; import com.google.protobuf.ByteString; import com.google.protobuf.Empty; +import com.google.protobuf.FieldMask; import com.google.pubsub.v1.CreateSchemaRequest; import com.google.pubsub.v1.DeleteSchemaRequest; import com.google.pubsub.v1.Encoding; @@ -559,6 +561,7 @@ public void setIamPolicyTest() throws Exception { Policy.newBuilder() .setVersion(351608024) .addAllBindings(new ArrayList()) + .addAllAuditConfigs(new ArrayList()) .setEtag(ByteString.EMPTY) .build(); mockIAMPolicy.addResponse(expectedResponse); @@ -567,6 +570,7 @@ public void setIamPolicyTest() throws Exception { SetIamPolicyRequest.newBuilder() .setResource(ProjectName.of("[PROJECT]").toString()) .setPolicy(Policy.newBuilder().build()) + .setUpdateMask(FieldMask.newBuilder().build()) .build(); Policy actualResponse = client.setIamPolicy(request); @@ -578,6 +582,7 @@ public void setIamPolicyTest() throws Exception { Assert.assertEquals(request.getResource(), actualRequest.getResource()); Assert.assertEquals(request.getPolicy(), actualRequest.getPolicy()); + Assert.assertEquals(request.getUpdateMask(), actualRequest.getUpdateMask()); Assert.assertTrue( channelProvider.isHeaderSent( ApiClientHeaderProvider.getDefaultApiClientHeaderKey(), @@ -594,6 +599,7 @@ public void setIamPolicyExceptionTest() throws Exception { SetIamPolicyRequest.newBuilder() .setResource(ProjectName.of("[PROJECT]").toString()) .setPolicy(Policy.newBuilder().build()) + .setUpdateMask(FieldMask.newBuilder().build()) .build(); client.setIamPolicy(request); Assert.fail("No exception raised"); @@ -608,6 +614,7 @@ public void getIamPolicyTest() throws Exception { Policy.newBuilder() .setVersion(351608024) .addAllBindings(new ArrayList()) + .addAllAuditConfigs(new ArrayList()) .setEtag(ByteString.EMPTY) .build(); mockIAMPolicy.addResponse(expectedResponse); diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java index 4fef8c98c..dbb87a03e 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java @@ -31,6 +31,7 @@ import com.google.api.gax.rpc.InvalidArgumentException; import com.google.api.gax.rpc.StatusCode; import com.google.common.collect.Lists; +import com.google.iam.v1.AuditConfig; import com.google.iam.v1.Binding; import com.google.iam.v1.GetIamPolicyRequest; import com.google.iam.v1.GetPolicyOptions; @@ -1679,6 +1680,7 @@ public void setIamPolicyTest() throws Exception { Policy.newBuilder() .setVersion(351608024) .addAllBindings(new ArrayList()) + .addAllAuditConfigs(new ArrayList()) .setEtag(ByteString.EMPTY) .build(); mockIAMPolicy.addResponse(expectedResponse); @@ -1687,6 +1689,7 @@ public void setIamPolicyTest() throws Exception { SetIamPolicyRequest.newBuilder() .setResource(ProjectName.of("[PROJECT]").toString()) .setPolicy(Policy.newBuilder().build()) + .setUpdateMask(FieldMask.newBuilder().build()) .build(); Policy actualResponse = client.setIamPolicy(request); @@ -1698,6 +1701,7 @@ public void setIamPolicyTest() throws Exception { Assert.assertEquals(request.getResource(), actualRequest.getResource()); Assert.assertEquals(request.getPolicy(), actualRequest.getPolicy()); + Assert.assertEquals(request.getUpdateMask(), actualRequest.getUpdateMask()); Assert.assertTrue( channelProvider.isHeaderSent( ApiClientHeaderProvider.getDefaultApiClientHeaderKey(), @@ -1714,6 +1718,7 @@ public void setIamPolicyExceptionTest() throws Exception { SetIamPolicyRequest.newBuilder() .setResource(ProjectName.of("[PROJECT]").toString()) .setPolicy(Policy.newBuilder().build()) + .setUpdateMask(FieldMask.newBuilder().build()) .build(); client.setIamPolicy(request); Assert.fail("No exception raised"); @@ -1728,6 +1733,7 @@ public void getIamPolicyTest() throws Exception { Policy.newBuilder() .setVersion(351608024) .addAllBindings(new ArrayList()) + .addAllAuditConfigs(new ArrayList()) .setEtag(ByteString.EMPTY) .build(); mockIAMPolicy.addResponse(expectedResponse); diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java index 02d6a5585..a3c4e4245 100644 --- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java +++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java @@ -28,6 +28,7 @@ import com.google.api.gax.rpc.ApiClientHeaderProvider; import com.google.api.gax.rpc.InvalidArgumentException; import com.google.common.collect.Lists; +import com.google.iam.v1.AuditConfig; import com.google.iam.v1.Binding; import com.google.iam.v1.GetIamPolicyRequest; import com.google.iam.v1.GetPolicyOptions; @@ -804,6 +805,7 @@ public void setIamPolicyTest() throws Exception { Policy.newBuilder() .setVersion(351608024) .addAllBindings(new ArrayList()) + .addAllAuditConfigs(new ArrayList()) .setEtag(ByteString.EMPTY) .build(); mockIAMPolicy.addResponse(expectedResponse); @@ -812,6 +814,7 @@ public void setIamPolicyTest() throws Exception { SetIamPolicyRequest.newBuilder() .setResource(ProjectName.of("[PROJECT]").toString()) .setPolicy(Policy.newBuilder().build()) + .setUpdateMask(FieldMask.newBuilder().build()) .build(); Policy actualResponse = client.setIamPolicy(request); @@ -823,6 +826,7 @@ public void setIamPolicyTest() throws Exception { Assert.assertEquals(request.getResource(), actualRequest.getResource()); Assert.assertEquals(request.getPolicy(), actualRequest.getPolicy()); + Assert.assertEquals(request.getUpdateMask(), actualRequest.getUpdateMask()); Assert.assertTrue( channelProvider.isHeaderSent( ApiClientHeaderProvider.getDefaultApiClientHeaderKey(), @@ -839,6 +843,7 @@ public void setIamPolicyExceptionTest() throws Exception { SetIamPolicyRequest.newBuilder() .setResource(ProjectName.of("[PROJECT]").toString()) .setPolicy(Policy.newBuilder().build()) + .setUpdateMask(FieldMask.newBuilder().build()) .build(); client.setIamPolicy(request); Assert.fail("No exception raised"); @@ -853,6 +858,7 @@ public void getIamPolicyTest() throws Exception { Policy.newBuilder() .setVersion(351608024) .addAllBindings(new ArrayList()) + .addAllAuditConfigs(new ArrayList()) .setEtag(ByteString.EMPTY) .build(); mockIAMPolicy.addResponse(expectedResponse); diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/AcknowledgeRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/AcknowledgeRequest.java index cf164758b..38f0841ad 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/AcknowledgeRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/AcknowledgeRequest.java @@ -100,6 +100,8 @@ private AcknowledgeRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSchemaRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSchemaRequest.java index e0a121785..9f2348931 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSchemaRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSchemaRequest.java @@ -110,6 +110,8 @@ private CreateSchemaRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequest.java index b37c9f64a..6140d5e45 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequest.java @@ -110,6 +110,8 @@ private CreateSnapshotRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -314,7 +316,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -351,7 +353,7 @@ public java.util.Map getLabelsMap() { @java.lang.Override public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -369,7 +371,7 @@ public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.Strin @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -1089,7 +1091,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -1127,7 +1129,7 @@ public java.util.Map getLabelsMap() { public java.lang.String getLabelsOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -1145,7 +1147,7 @@ public java.lang.String getLabelsOrDefault( @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -1170,7 +1172,7 @@ public Builder clearLabels() { */ public Builder removeLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } internalGetMutableLabels().getMutableMap().remove(key); return this; @@ -1192,11 +1194,12 @@ public java.util.Map getMutableLabels() { */ public Builder putLabels(java.lang.String key, java.lang.String value) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } if (value == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map value"); } + internalGetMutableLabels().getMutableMap().put(key, value); return this; } diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequestOrBuilder.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequestOrBuilder.java index b8e0ef2f9..647f8506e 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequestOrBuilder.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/CreateSnapshotRequestOrBuilder.java @@ -153,7 +153,12 @@ public interface CreateSnapshotRequestOrBuilder * * map<string, string> labels = 3; */ - java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue); + + /* nullable */ + java.lang.String getLabelsOrDefault( + java.lang.String key, + /* nullable */ + java.lang.String defaultValue); /** * * diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeadLetterPolicy.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeadLetterPolicy.java index e543187a8..018a74139 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeadLetterPolicy.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeadLetterPolicy.java @@ -96,6 +96,8 @@ private DeadLetterPolicy( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSchemaRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSchemaRequest.java index 69a7b241f..84747a7ae 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSchemaRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSchemaRequest.java @@ -88,6 +88,8 @@ private DeleteSchemaRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSnapshotRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSnapshotRequest.java index b5529abe6..8113bad2e 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSnapshotRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSnapshotRequest.java @@ -88,6 +88,8 @@ private DeleteSnapshotRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSubscriptionRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSubscriptionRequest.java index e4ca9d30f..90bb58433 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSubscriptionRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteSubscriptionRequest.java @@ -88,6 +88,8 @@ private DeleteSubscriptionRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteTopicRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteTopicRequest.java index db0ba9fbc..5b3a9e2cb 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteTopicRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DeleteTopicRequest.java @@ -88,6 +88,8 @@ private DeleteTopicRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionRequest.java index 8e56f9e90..80e218214 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionRequest.java @@ -88,6 +88,8 @@ private DetachSubscriptionRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionResponse.java index a473832a2..425eff37f 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/DetachSubscriptionResponse.java @@ -80,6 +80,8 @@ private DetachSubscriptionResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ExpirationPolicy.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ExpirationPolicy.java index 5c88243ee..e9ed8b2f7 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ExpirationPolicy.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ExpirationPolicy.java @@ -94,6 +94,8 @@ private ExpirationPolicy( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSchemaRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSchemaRequest.java index 9eff93962..9a1f22ba3 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSchemaRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSchemaRequest.java @@ -96,6 +96,8 @@ private GetSchemaRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSnapshotRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSnapshotRequest.java index 2cdf4abfe..e40c7802e 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSnapshotRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSnapshotRequest.java @@ -88,6 +88,8 @@ private GetSnapshotRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSubscriptionRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSubscriptionRequest.java index 1c9992ee8..0a0122d3c 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSubscriptionRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetSubscriptionRequest.java @@ -88,6 +88,8 @@ private GetSubscriptionRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetTopicRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetTopicRequest.java index 0d3e779c5..43e15615e 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetTopicRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/GetTopicRequest.java @@ -88,6 +88,8 @@ private GetTopicRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasRequest.java index 0c8e4883d..9e75e40f4 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasRequest.java @@ -109,6 +109,8 @@ private ListSchemasRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasResponse.java index 27f189db1..d7acc8862 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSchemasResponse.java @@ -100,6 +100,8 @@ private ListSchemasResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsRequest.java index 6958265a6..c23abd212 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsRequest.java @@ -101,6 +101,8 @@ private ListSnapshotsRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsResponse.java index d7747b6dd..8d44aa126 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSnapshotsResponse.java @@ -100,6 +100,8 @@ private ListSnapshotsResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsRequest.java index 7599fbf28..d7db4c0ea 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsRequest.java @@ -101,6 +101,8 @@ private ListSubscriptionsRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsResponse.java index 614895dde..237045561 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListSubscriptionsResponse.java @@ -100,6 +100,8 @@ private ListSubscriptionsResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsRequest.java index 9ba97ba5a..17a383573 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsRequest.java @@ -101,6 +101,8 @@ private ListTopicSnapshotsRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsResponse.java index c098cc39a..ea2130886 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSnapshotsResponse.java @@ -100,6 +100,8 @@ private ListTopicSnapshotsResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsRequest.java index d8a5d96fd..ae8c370d6 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsRequest.java @@ -101,6 +101,8 @@ private ListTopicSubscriptionsRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsResponse.java index c81e4fe62..16bfc8bfe 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicSubscriptionsResponse.java @@ -101,6 +101,8 @@ private ListTopicSubscriptionsResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsRequest.java index 17f59fd63..cbb0a15ed 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsRequest.java @@ -101,6 +101,8 @@ private ListTopicsRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsResponse.java index 0af6dd447..ba9cb473c 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ListTopicsResponse.java @@ -100,6 +100,8 @@ private ListTopicsResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/MessageStoragePolicy.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/MessageStoragePolicy.java index b03bda501..c02987c8e 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/MessageStoragePolicy.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/MessageStoragePolicy.java @@ -92,6 +92,8 @@ private MessageStoragePolicy( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyAckDeadlineRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyAckDeadlineRequest.java index 30831833d..f3c067c7d 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyAckDeadlineRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyAckDeadlineRequest.java @@ -105,6 +105,8 @@ private ModifyAckDeadlineRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyPushConfigRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyPushConfigRequest.java index 8833432d8..f3d869b47 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyPushConfigRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ModifyPushConfigRequest.java @@ -103,6 +103,8 @@ private ModifyPushConfigRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishRequest.java index 3c1a2c428..4d0732319 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishRequest.java @@ -101,6 +101,8 @@ private PublishRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishResponse.java index 750a74022..25caaf8bc 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PublishResponse.java @@ -92,6 +92,8 @@ private PublishResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessage.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessage.java index bda025a24..f9243a265 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessage.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessage.java @@ -140,6 +140,8 @@ private PubsubMessage( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -232,7 +234,7 @@ public int getAttributesCount() { @java.lang.Override public boolean containsAttributes(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetAttributes().getMap().containsKey(key); } @@ -272,7 +274,7 @@ public java.util.Map getAttributesMap() { public java.lang.String getAttributesOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -291,7 +293,7 @@ public java.lang.String getAttributesOrDefault( @java.lang.Override public java.lang.String getAttributesOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); if (!map.containsKey(key)) { @@ -989,7 +991,7 @@ public int getAttributesCount() { @java.lang.Override public boolean containsAttributes(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetAttributes().getMap().containsKey(key); } @@ -1029,7 +1031,7 @@ public java.util.Map getAttributesMap() { public java.lang.String getAttributesOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -1048,7 +1050,7 @@ public java.lang.String getAttributesOrDefault( @java.lang.Override public java.lang.String getAttributesOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); if (!map.containsKey(key)) { @@ -1074,7 +1076,7 @@ public Builder clearAttributes() { */ public Builder removeAttributes(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } internalGetMutableAttributes().getMutableMap().remove(key); return this; @@ -1097,11 +1099,12 @@ public java.util.Map getMutableAttributes() */ public Builder putAttributes(java.lang.String key, java.lang.String value) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } if (value == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map value"); } + internalGetMutableAttributes().getMutableMap().put(key, value); return this; } diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessageOrBuilder.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessageOrBuilder.java index a81cdd8e2..c06beec09 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessageOrBuilder.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PubsubMessageOrBuilder.java @@ -87,7 +87,12 @@ public interface PubsubMessageOrBuilder * * map<string, string> attributes = 2; */ - java.lang.String getAttributesOrDefault(java.lang.String key, java.lang.String defaultValue); + + /* nullable */ + java.lang.String getAttributesOrDefault( + java.lang.String key, + /* nullable */ + java.lang.String defaultValue); /** * * diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequest.java index 097ffd48d..81ff04645 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequest.java @@ -98,6 +98,8 @@ private PullRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -192,6 +194,8 @@ public com.google.protobuf.ByteString getSubscriptionBytes() { * bool return_immediately = 2 [deprecated = true, (.google.api.field_behavior) = OPTIONAL]; * * + * @deprecated google.pubsub.v1.PullRequest.return_immediately is deprecated. See + * google/pubsub/v1/pubsub.proto;l=1035 * @return The returnImmediately. */ @java.lang.Override @@ -702,6 +706,8 @@ public Builder setSubscriptionBytes(com.google.protobuf.ByteString value) { * bool return_immediately = 2 [deprecated = true, (.google.api.field_behavior) = OPTIONAL]; * * + * @deprecated google.pubsub.v1.PullRequest.return_immediately is deprecated. See + * google/pubsub/v1/pubsub.proto;l=1035 * @return The returnImmediately. */ @java.lang.Override @@ -726,6 +732,8 @@ public boolean getReturnImmediately() { * bool return_immediately = 2 [deprecated = true, (.google.api.field_behavior) = OPTIONAL]; * * + * @deprecated google.pubsub.v1.PullRequest.return_immediately is deprecated. See + * google/pubsub/v1/pubsub.proto;l=1035 * @param value The returnImmediately to set. * @return This builder for chaining. */ @@ -753,6 +761,8 @@ public Builder setReturnImmediately(boolean value) { * bool return_immediately = 2 [deprecated = true, (.google.api.field_behavior) = OPTIONAL]; * * + * @deprecated google.pubsub.v1.PullRequest.return_immediately is deprecated. See + * google/pubsub/v1/pubsub.proto;l=1035 * @return This builder for chaining. */ @java.lang.Deprecated diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequestOrBuilder.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequestOrBuilder.java index eb88907a9..962ab3955 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequestOrBuilder.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullRequestOrBuilder.java @@ -70,6 +70,8 @@ public interface PullRequestOrBuilder * bool return_immediately = 2 [deprecated = true, (.google.api.field_behavior) = OPTIONAL]; * * + * @deprecated google.pubsub.v1.PullRequest.return_immediately is deprecated. See + * google/pubsub/v1/pubsub.proto;l=1035 * @return The returnImmediately. */ @java.lang.Deprecated diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullResponse.java index bca433c65..f0482b4c3 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PullResponse.java @@ -93,6 +93,8 @@ private PullResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfig.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfig.java index 74e82b909..c551db41d 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfig.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfig.java @@ -122,6 +122,8 @@ private PushConfig( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -307,6 +309,8 @@ private OidcToken( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -1240,7 +1244,7 @@ public int getAttributesCount() { @java.lang.Override public boolean containsAttributes(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetAttributes().getMap().containsKey(key); } @@ -1306,7 +1310,7 @@ public java.util.Map getAttributesMap() { public java.lang.String getAttributesOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -1338,7 +1342,7 @@ public java.lang.String getAttributesOrDefault( @java.lang.Override public java.lang.String getAttributesOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); if (!map.containsKey(key)) { @@ -1982,7 +1986,7 @@ public int getAttributesCount() { @java.lang.Override public boolean containsAttributes(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetAttributes().getMap().containsKey(key); } @@ -2048,7 +2052,7 @@ public java.util.Map getAttributesMap() { public java.lang.String getAttributesOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -2080,7 +2084,7 @@ public java.lang.String getAttributesOrDefault( @java.lang.Override public java.lang.String getAttributesOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetAttributes().getMap(); if (!map.containsKey(key)) { @@ -2119,7 +2123,7 @@ public Builder clearAttributes() { */ public Builder removeAttributes(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } internalGetMutableAttributes().getMutableMap().remove(key); return this; @@ -2155,11 +2159,12 @@ public java.util.Map getMutableAttributes() */ public Builder putAttributes(java.lang.String key, java.lang.String value) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } if (value == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map value"); } + internalGetMutableAttributes().getMutableMap().put(key, value); return this; } @@ -2309,8 +2314,9 @@ public Builder mergeOidcToken(com.google.pubsub.v1.PushConfig.OidcToken value) { } else { if (authenticationMethodCase_ == 3) { oidcTokenBuilder_.mergeFrom(value); + } else { + oidcTokenBuilder_.setMessage(value); } - oidcTokenBuilder_.setMessage(value); } authenticationMethodCase_ = 3; return this; diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfigOrBuilder.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfigOrBuilder.java index 091a066f9..6499b5346 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfigOrBuilder.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/PushConfigOrBuilder.java @@ -152,7 +152,12 @@ public interface PushConfigOrBuilder * * map<string, string> attributes = 2; */ - java.lang.String getAttributesOrDefault(java.lang.String key, java.lang.String defaultValue); + + /* nullable */ + java.lang.String getAttributesOrDefault( + java.lang.String key, + /* nullable */ + java.lang.String defaultValue); /** * * diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ReceivedMessage.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ReceivedMessage.java index 84c1a8242..7145c9e05 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ReceivedMessage.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ReceivedMessage.java @@ -108,6 +108,8 @@ private ReceivedMessage( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/RetryPolicy.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/RetryPolicy.java index 4b6a1934e..51f5b8e47 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/RetryPolicy.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/RetryPolicy.java @@ -116,6 +116,8 @@ private RetryPolicy( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Schema.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Schema.java index c157eb4ba..9dfc5bca4 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Schema.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Schema.java @@ -104,6 +104,8 @@ private Schema( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaSettings.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaSettings.java index ee21a2fe6..38a3b711e 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaSettings.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaSettings.java @@ -96,6 +96,8 @@ private SchemaSettings( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekRequest.java index a10126f4a..813c4f2d4 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekRequest.java @@ -110,6 +110,8 @@ private SeekRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -1058,8 +1060,9 @@ public Builder mergeTime(com.google.protobuf.Timestamp value) { } else { if (targetCase_ == 2) { timeBuilder_.mergeFrom(value); + } else { + timeBuilder_.setMessage(value); } - timeBuilder_.setMessage(value); } targetCase_ = 2; return this; diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekResponse.java index 3ced96f97..fe0aff80d 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SeekResponse.java @@ -79,6 +79,8 @@ private SeekResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Snapshot.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Snapshot.java index 765007468..927b203e8 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Snapshot.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Snapshot.java @@ -129,6 +129,8 @@ private Snapshot( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -370,7 +372,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -407,7 +409,7 @@ public java.util.Map getLabelsMap() { @java.lang.Override public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -425,7 +427,7 @@ public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.Strin @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -1350,7 +1352,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -1388,7 +1390,7 @@ public java.util.Map getLabelsMap() { public java.lang.String getLabelsOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -1406,7 +1408,7 @@ public java.lang.String getLabelsOrDefault( @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -1431,7 +1433,7 @@ public Builder clearLabels() { */ public Builder removeLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } internalGetMutableLabels().getMutableMap().remove(key); return this; @@ -1453,11 +1455,12 @@ public java.util.Map getMutableLabels() { */ public Builder putLabels(java.lang.String key, java.lang.String value) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } if (value == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map value"); } + internalGetMutableLabels().getMutableMap().put(key, value); return this; } diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotOrBuilder.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotOrBuilder.java index c8d3ed3ad..74761e4ad 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotOrBuilder.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotOrBuilder.java @@ -181,7 +181,12 @@ public interface SnapshotOrBuilder * * map<string, string> labels = 4; */ - java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue); + + /* nullable */ + java.lang.String getLabelsOrDefault( + java.lang.String key, + /* nullable */ + java.lang.String defaultValue); /** * * diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullRequest.java index a5b507540..fbeb6c13b 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullRequest.java @@ -160,6 +160,8 @@ private StreamingPullRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullResponse.java index 82ca2a8a9..0ea49f678 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/StreamingPullResponse.java @@ -149,6 +149,8 @@ private StreamingPullResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -434,6 +436,8 @@ private AcknowledgeConfirmation( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -1838,6 +1842,8 @@ private ModifyAckDeadlineConfirmation( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -2893,6 +2899,8 @@ private SubscriptionProperties( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Subscription.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Subscription.java index 0543e587d..c05fea07e 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Subscription.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Subscription.java @@ -235,6 +235,8 @@ private Subscription( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -595,7 +597,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -632,7 +634,7 @@ public java.util.Map getLabelsMap() { @java.lang.Override public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -650,7 +652,7 @@ public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.Strin @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -2580,7 +2582,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -2618,7 +2620,7 @@ public java.util.Map getLabelsMap() { public java.lang.String getLabelsOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -2636,7 +2638,7 @@ public java.lang.String getLabelsOrDefault( @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -2661,7 +2663,7 @@ public Builder clearLabels() { */ public Builder removeLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } internalGetMutableLabels().getMutableMap().remove(key); return this; @@ -2683,11 +2685,12 @@ public java.util.Map getMutableLabels() { */ public Builder putLabels(java.lang.String key, java.lang.String value) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } if (value == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map value"); } + internalGetMutableLabels().getMutableMap().put(key, value); return this; } diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionOrBuilder.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionOrBuilder.java index 64a0a771c..2c85e903f 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionOrBuilder.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionOrBuilder.java @@ -275,7 +275,12 @@ public interface SubscriptionOrBuilder * * map<string, string> labels = 9; */ - java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue); + + /* nullable */ + java.lang.String getLabelsOrDefault( + java.lang.String key, + /* nullable */ + java.lang.String defaultValue); /** * * diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Topic.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Topic.java index 1682c2329..6361c9e27 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Topic.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/Topic.java @@ -162,6 +162,8 @@ private Topic( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -291,7 +293,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -328,7 +330,7 @@ public java.util.Map getLabelsMap() { @java.lang.Override public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -346,7 +348,7 @@ public java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.Strin @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -1239,7 +1241,7 @@ public int getLabelsCount() { @java.lang.Override public boolean containsLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } return internalGetLabels().getMap().containsKey(key); } @@ -1277,7 +1279,7 @@ public java.util.Map getLabelsMap() { public java.lang.String getLabelsOrDefault( java.lang.String key, java.lang.String defaultValue) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); return map.containsKey(key) ? map.get(key) : defaultValue; @@ -1295,7 +1297,7 @@ public java.lang.String getLabelsOrDefault( @java.lang.Override public java.lang.String getLabelsOrThrow(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } java.util.Map map = internalGetLabels().getMap(); if (!map.containsKey(key)) { @@ -1320,7 +1322,7 @@ public Builder clearLabels() { */ public Builder removeLabels(java.lang.String key) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } internalGetMutableLabels().getMutableMap().remove(key); return this; @@ -1342,11 +1344,12 @@ public java.util.Map getMutableLabels() { */ public Builder putLabels(java.lang.String key, java.lang.String value) { if (key == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map key"); } if (value == null) { - throw new java.lang.NullPointerException(); + throw new NullPointerException("map value"); } + internalGetMutableLabels().getMutableMap().put(key, value); return this; } diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicOrBuilder.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicOrBuilder.java index e9e3dae79..cbc1957b4 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicOrBuilder.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicOrBuilder.java @@ -104,7 +104,12 @@ public interface TopicOrBuilder * * map<string, string> labels = 2; */ - java.lang.String getLabelsOrDefault(java.lang.String key, java.lang.String defaultValue); + + /* nullable */ + java.lang.String getLabelsOrDefault( + java.lang.String key, + /* nullable */ + java.lang.String defaultValue); /** * * diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSnapshotRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSnapshotRequest.java index 078d0d1be..1925b6c4f 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSnapshotRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSnapshotRequest.java @@ -109,6 +109,8 @@ private UpdateSnapshotRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSubscriptionRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSubscriptionRequest.java index ffed68bcf..0e9110799 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSubscriptionRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateSubscriptionRequest.java @@ -109,6 +109,8 @@ private UpdateSubscriptionRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateTopicRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateTopicRequest.java index 7458cb8ad..51fe720e6 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateTopicRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/UpdateTopicRequest.java @@ -108,6 +108,8 @@ private UpdateTopicRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageRequest.java index c5bdcde47..b58b83ebc 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageRequest.java @@ -124,6 +124,8 @@ private ValidateMessageRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { @@ -1225,8 +1227,9 @@ public Builder mergeSchema(com.google.pubsub.v1.Schema value) { } else { if (schemaSpecCase_ == 3) { schemaBuilder_.mergeFrom(value); + } else { + schemaBuilder_.setMessage(value); } - schemaBuilder_.setMessage(value); } schemaSpecCase_ = 3; return this; diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageResponse.java index d86904cf0..f734b6925 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateMessageResponse.java @@ -80,6 +80,8 @@ private ValidateMessageResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaRequest.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaRequest.java index 0e62b8478..5aa2bf5dd 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaRequest.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaRequest.java @@ -102,6 +102,8 @@ private ValidateSchemaRequest( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaResponse.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaResponse.java index 881b35e95..e7755cca3 100644 --- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaResponse.java +++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ValidateSchemaResponse.java @@ -80,6 +80,8 @@ private ValidateSchemaResponse( } } catch (com.google.protobuf.InvalidProtocolBufferException e) { throw e.setUnfinishedMessage(this); + } catch (com.google.protobuf.UninitializedMessageException e) { + throw e.asInvalidProtocolBufferException().setUnfinishedMessage(this); } catch (java.io.IOException e) { throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); } finally { From 66d356f80747fbad69c21fcec6da39f177d9dcab Mon Sep 17 00:00:00 2001 From: "gcf-owl-bot[bot]" <78513119+gcf-owl-bot[bot]@users.noreply.github.com> Date: Thu, 5 May 2022 20:48:14 +0000 Subject: [PATCH 49/52] chore(deps): upgrade gapic-generator-java to 2.7.0 and update gax-java to 2.16.0 (#1107) - [ ] Regenerate this pull request now. PiperOrigin-RevId: 446250659 Source-Link: https://github.com/googleapis/googleapis/commit/dc4ef314fecf1b00833e78288cf2eb4d2b165ad1 Source-Link: https://github.com/googleapis/googleapis-gen/commit/5fdda4ddfbab87998e77e4a453e0ff87986d2db8 Copy-Tag: eyJwIjoiLmdpdGh1Yi8uT3dsQm90LnlhbWwiLCJoIjoiNWZkZGE0ZGRmYmFiODc5OThlNzdlNGE0NTNlMGZmODc5ODZkMmRiOCJ9 --- .../cloud/pubsub/v1/SchemaServiceClient.java | 66 +++++++- .../pubsub/v1/SchemaServiceSettings.java | 4 +- .../pubsub/v1/SubscriptionAdminClient.java | 146 +++++++++++++++++- .../pubsub/v1/SubscriptionAdminSettings.java | 4 +- .../cloud/pubsub/v1/TopicAdminClient.java | 90 ++++++++++- .../cloud/pubsub/v1/TopicAdminSettings.java | 4 +- .../google/cloud/pubsub/v1/package-info.java | 8 +- .../v1/stub/GrpcPublisherCallableFactory.java | 2 +- .../pubsub/v1/stub/GrpcPublisherStub.java | 2 +- .../GrpcSchemaServiceCallableFactory.java | 2 +- .../pubsub/v1/stub/GrpcSchemaServiceStub.java | 2 +- .../stub/GrpcSubscriberCallableFactory.java | 2 +- .../pubsub/v1/stub/GrpcSubscriberStub.java | 2 +- .../cloud/pubsub/v1/stub/PublisherStub.java | 2 +- .../pubsub/v1/stub/PublisherStubSettings.java | 4 +- .../pubsub/v1/stub/SchemaServiceStub.java | 2 +- .../v1/stub/SchemaServiceStubSettings.java | 4 +- .../cloud/pubsub/v1/stub/SubscriberStub.java | 2 +- .../v1/stub/SubscriberStubSettings.java | 4 +- .../google/cloud/pubsub/v1/MockIAMPolicy.java | 2 +- .../cloud/pubsub/v1/MockIAMPolicyImpl.java | 2 +- .../google/cloud/pubsub/v1/MockPublisher.java | 2 +- .../cloud/pubsub/v1/MockPublisherImpl.java | 2 +- .../cloud/pubsub/v1/MockSchemaService.java | 2 +- .../pubsub/v1/MockSchemaServiceImpl.java | 2 +- .../cloud/pubsub/v1/MockSubscriber.java | 2 +- .../cloud/pubsub/v1/MockSubscriberImpl.java | 2 +- .../pubsub/v1/SchemaServiceClientTest.java | 2 +- .../v1/SubscriptionAdminClientTest.java | 2 +- .../cloud/pubsub/v1/TopicAdminClientTest.java | 2 +- .../com/google/pubsub/v1/ProjectName.java | 2 +- .../java/com/google/pubsub/v1/SchemaName.java | 2 +- .../com/google/pubsub/v1/SnapshotName.java | 2 +- .../google/pubsub/v1/SubscriptionName.java | 2 +- .../java/com/google/pubsub/v1/TopicName.java | 2 +- 35 files changed, 349 insertions(+), 35 deletions(-) diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java index fd0f238e2..3ab408e71 100644 --- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java +++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceClient.java @@ -1,5 +1,5 @@ /* - * Copyright 2021 Google LLC + * Copyright 2022 Google LLC * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -59,6 +59,8 @@ * calls that map to API methods. Sample code to get started: * *
{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
  *   ProjectName parent = ProjectName.of("[PROJECT]");
  *   Schema schema = Schema.newBuilder().build();
@@ -96,6 +98,8 @@
  * 

To customize credentials: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SchemaServiceSettings schemaServiceSettings =
  *     SchemaServiceSettings.newBuilder()
  *         .setCredentialsProvider(FixedCredentialsProvider.create(myCredentials))
@@ -106,6 +110,8 @@
  * 

To customize the endpoint: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SchemaServiceSettings schemaServiceSettings =
  *     SchemaServiceSettings.newBuilder().setEndpoint(myEndpoint).build();
  * SchemaServiceClient schemaServiceClient = SchemaServiceClient.create(schemaServiceSettings);
@@ -173,6 +179,8 @@ public SchemaServiceStub getStub() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ProjectName parent = ProjectName.of("[PROJECT]");
    *   Schema schema = Schema.newBuilder().build();
@@ -209,6 +217,8 @@ public final Schema createSchema(ProjectName parent, Schema schema, String schem
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   String parent = ProjectName.of("[PROJECT]").toString();
    *   Schema schema = Schema.newBuilder().build();
@@ -245,6 +255,8 @@ public final Schema createSchema(String parent, Schema schema, String schemaId)
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   CreateSchemaRequest request =
    *       CreateSchemaRequest.newBuilder()
@@ -270,6 +282,8 @@ public final Schema createSchema(CreateSchemaRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   CreateSchemaRequest request =
    *       CreateSchemaRequest.newBuilder()
@@ -294,6 +308,8 @@ public final UnaryCallable createSchemaCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   SchemaName name = SchemaName.of("[PROJECT]", "[SCHEMA]");
    *   Schema response = schemaServiceClient.getSchema(name);
@@ -317,6 +333,8 @@ public final Schema getSchema(SchemaName name) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   String name = SchemaName.of("[PROJECT]", "[SCHEMA]").toString();
    *   Schema response = schemaServiceClient.getSchema(name);
@@ -339,6 +357,8 @@ public final Schema getSchema(String name) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   GetSchemaRequest request =
    *       GetSchemaRequest.newBuilder()
@@ -363,6 +383,8 @@ public final Schema getSchema(GetSchemaRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   GetSchemaRequest request =
    *       GetSchemaRequest.newBuilder()
@@ -386,6 +408,8 @@ public final UnaryCallable getSchemaCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ProjectName parent = ProjectName.of("[PROJECT]");
    *   for (Schema element : schemaServiceClient.listSchemas(parent).iterateAll()) {
@@ -413,6 +437,8 @@ public final ListSchemasPagedResponse listSchemas(ProjectName parent) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   String parent = ProjectName.of("[PROJECT]").toString();
    *   for (Schema element : schemaServiceClient.listSchemas(parent).iterateAll()) {
@@ -437,6 +463,8 @@ public final ListSchemasPagedResponse listSchemas(String parent) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ListSchemasRequest request =
    *       ListSchemasRequest.newBuilder()
@@ -465,6 +493,8 @@ public final ListSchemasPagedResponse listSchemas(ListSchemasRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ListSchemasRequest request =
    *       ListSchemasRequest.newBuilder()
@@ -493,6 +523,8 @@ public final ListSchemasPagedResponse listSchemas(ListSchemasRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ListSchemasRequest request =
    *       ListSchemasRequest.newBuilder()
@@ -527,6 +559,8 @@ public final UnaryCallable listSchemasC
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   SchemaName name = SchemaName.of("[PROJECT]", "[SCHEMA]");
    *   schemaServiceClient.deleteSchema(name);
@@ -550,6 +584,8 @@ public final void deleteSchema(SchemaName name) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   String name = SchemaName.of("[PROJECT]", "[SCHEMA]").toString();
    *   schemaServiceClient.deleteSchema(name);
@@ -572,6 +608,8 @@ public final void deleteSchema(String name) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   DeleteSchemaRequest request =
    *       DeleteSchemaRequest.newBuilder()
@@ -595,6 +633,8 @@ public final void deleteSchema(DeleteSchemaRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   DeleteSchemaRequest request =
    *       DeleteSchemaRequest.newBuilder()
@@ -617,6 +657,8 @@ public final UnaryCallable deleteSchemaCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ProjectName parent = ProjectName.of("[PROJECT]");
    *   Schema schema = Schema.newBuilder().build();
@@ -645,6 +687,8 @@ public final ValidateSchemaResponse validateSchema(ProjectName parent, Schema sc
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   String parent = ProjectName.of("[PROJECT]").toString();
    *   Schema schema = Schema.newBuilder().build();
@@ -670,6 +714,8 @@ public final ValidateSchemaResponse validateSchema(String parent, Schema schema)
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ValidateSchemaRequest request =
    *       ValidateSchemaRequest.newBuilder()
@@ -694,6 +740,8 @@ public final ValidateSchemaResponse validateSchema(ValidateSchemaRequest request
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ValidateSchemaRequest request =
    *       ValidateSchemaRequest.newBuilder()
@@ -719,6 +767,8 @@ public final ValidateSchemaResponse validateSchema(ValidateSchemaRequest request
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ValidateMessageRequest request =
    *       ValidateMessageRequest.newBuilder()
@@ -744,6 +794,8 @@ public final ValidateMessageResponse validateMessage(ValidateMessageRequest requ
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   ValidateMessageRequest request =
    *       ValidateMessageRequest.newBuilder()
@@ -772,6 +824,8 @@ public final ValidateMessageResponse validateMessage(ValidateMessageRequest requ
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   SetIamPolicyRequest request =
    *       SetIamPolicyRequest.newBuilder()
@@ -799,6 +853,8 @@ public final Policy setIamPolicy(SetIamPolicyRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   SetIamPolicyRequest request =
    *       SetIamPolicyRequest.newBuilder()
@@ -824,6 +880,8 @@ public final UnaryCallable setIamPolicyCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   GetIamPolicyRequest request =
    *       GetIamPolicyRequest.newBuilder()
@@ -849,6 +907,8 @@ public final Policy getIamPolicy(GetIamPolicyRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   GetIamPolicyRequest request =
    *       GetIamPolicyRequest.newBuilder()
@@ -877,6 +937,8 @@ public final UnaryCallable getIamPolicyCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   TestIamPermissionsRequest request =
    *       TestIamPermissionsRequest.newBuilder()
@@ -906,6 +968,8 @@ public final TestIamPermissionsResponse testIamPermissions(TestIamPermissionsReq
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
    *   TestIamPermissionsRequest request =
    *       TestIamPermissionsRequest.newBuilder()
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceSettings.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceSettings.java
index e8c50371c..597c33bc6 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceSettings.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SchemaServiceSettings.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -68,6 +68,8 @@
  * 

For example, to set the total timeout of createSchema to 30 seconds: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SchemaServiceSettings.Builder schemaServiceSettingsBuilder = SchemaServiceSettings.newBuilder();
  * schemaServiceSettingsBuilder
  *     .createSchemaSettings()
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java
index 8f11330eb..9b48ca018 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminClient.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -80,6 +80,8 @@
  * calls that map to API methods. Sample code to get started:
  *
  * 
{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
  *   SubscriptionName name = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
  *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
@@ -120,6 +122,8 @@
  * 

To customize credentials: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SubscriptionAdminSettings subscriptionAdminSettings =
  *     SubscriptionAdminSettings.newBuilder()
  *         .setCredentialsProvider(FixedCredentialsProvider.create(myCredentials))
@@ -131,6 +135,8 @@
  * 

To customize the endpoint: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SubscriptionAdminSettings subscriptionAdminSettings =
  *     SubscriptionAdminSettings.newBuilder().setEndpoint(myEndpoint).build();
  * SubscriptionAdminClient subscriptionAdminClient =
@@ -208,6 +214,8 @@ public SubscriberStub getStub() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName name = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
@@ -274,6 +282,8 @@ public final Subscription createSubscription(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName name = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   String topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
@@ -340,6 +350,8 @@ public final Subscription createSubscription(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String name = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
@@ -406,6 +418,8 @@ public final Subscription createSubscription(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String name = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   String topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
@@ -472,6 +486,8 @@ public final Subscription createSubscription(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   Subscription request =
    *       Subscription.newBuilder()
@@ -728,6 +744,8 @@ public final Subscription createSubscription(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   Subscription request =
    *       Subscription.newBuilder()
@@ -765,6 +783,8 @@ public final UnaryCallable createSubscriptionCallabl
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   Subscription response = subscriptionAdminClient.getSubscription(subscription);
@@ -790,6 +810,8 @@ public final Subscription getSubscription(SubscriptionName subscription) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   Subscription response = subscriptionAdminClient.getSubscription(subscription);
@@ -813,6 +835,8 @@ public final Subscription getSubscription(String subscription) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   GetSubscriptionRequest request =
    *       GetSubscriptionRequest.newBuilder()
@@ -863,6 +887,8 @@ public final Subscription getSubscription(ProjectSubscriptionName subscription)
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   GetSubscriptionRequest request =
    *       GetSubscriptionRequest.newBuilder()
@@ -887,6 +913,8 @@ public final UnaryCallable getSubscription
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   UpdateSubscriptionRequest request =
    *       UpdateSubscriptionRequest.newBuilder()
@@ -912,6 +940,8 @@ public final Subscription updateSubscription(UpdateSubscriptionRequest request)
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   UpdateSubscriptionRequest request =
    *       UpdateSubscriptionRequest.newBuilder()
@@ -936,6 +966,8 @@ public final UnaryCallable updateSubscr
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ProjectName project = ProjectName.of("[PROJECT]");
    *   for (Subscription element : subscriptionAdminClient.listSubscriptions(project).iterateAll()) {
@@ -963,6 +995,8 @@ public final ListSubscriptionsPagedResponse listSubscriptions(ProjectName projec
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String project = ProjectName.of("[PROJECT]").toString();
    *   for (Subscription element : subscriptionAdminClient.listSubscriptions(project).iterateAll()) {
@@ -988,6 +1022,8 @@ public final ListSubscriptionsPagedResponse listSubscriptions(String project) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ListSubscriptionsRequest request =
    *       ListSubscriptionsRequest.newBuilder()
@@ -1015,6 +1051,8 @@ public final ListSubscriptionsPagedResponse listSubscriptions(ListSubscriptionsR
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ListSubscriptionsRequest request =
    *       ListSubscriptionsRequest.newBuilder()
@@ -1043,6 +1081,8 @@ public final ListSubscriptionsPagedResponse listSubscriptions(ListSubscriptionsR
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ListSubscriptionsRequest request =
    *       ListSubscriptionsRequest.newBuilder()
@@ -1081,6 +1121,8 @@ public final ListSubscriptionsPagedResponse listSubscriptions(ListSubscriptionsR
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   subscriptionAdminClient.deleteSubscription(subscription);
@@ -1109,6 +1151,8 @@ public final void deleteSubscription(SubscriptionName subscription) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   subscriptionAdminClient.deleteSubscription(subscription);
@@ -1135,6 +1179,8 @@ public final void deleteSubscription(String subscription) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   DeleteSubscriptionRequest request =
    *       DeleteSubscriptionRequest.newBuilder()
@@ -1191,6 +1237,8 @@ public final void deleteSubscription(ProjectSubscriptionName subscription) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   DeleteSubscriptionRequest request =
    *       DeleteSubscriptionRequest.newBuilder()
@@ -1217,6 +1265,8 @@ public final UnaryCallable deleteSubscriptionC
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   List ackIds = new ArrayList<>();
@@ -1258,6 +1308,8 @@ public final void modifyAckDeadline(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   List ackIds = new ArrayList<>();
@@ -1299,6 +1351,8 @@ public final void modifyAckDeadline(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ModifyAckDeadlineRequest request =
    *       ModifyAckDeadlineRequest.newBuilder()
@@ -1370,6 +1424,8 @@ final void modifyAckDeadline(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ModifyAckDeadlineRequest request =
    *       ModifyAckDeadlineRequest.newBuilder()
@@ -1399,6 +1455,8 @@ public final UnaryCallable modifyAckDeadlineCal
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   List ackIds = new ArrayList<>();
@@ -1432,6 +1490,8 @@ public final void acknowledge(SubscriptionName subscription, List ackIds
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   List ackIds = new ArrayList<>();
@@ -1462,6 +1522,8 @@ public final void acknowledge(String subscription, List ackIds) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   AcknowledgeRequest request =
    *       AcknowledgeRequest.newBuilder()
@@ -1525,6 +1587,8 @@ public final void acknowledge(ProjectSubscriptionName subscription, List
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   AcknowledgeRequest request =
    *       AcknowledgeRequest.newBuilder()
@@ -1549,6 +1613,8 @@ public final UnaryCallable acknowledgeCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   int maxMessages = 496131527;
@@ -1579,6 +1645,8 @@ public final PullResponse pull(SubscriptionName subscription, int maxMessages) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   int maxMessages = 496131527;
@@ -1606,6 +1674,8 @@ public final PullResponse pull(String subscription, int maxMessages) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   boolean returnImmediately = true;
@@ -1646,6 +1716,8 @@ public final PullResponse pull(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   boolean returnImmediately = true;
@@ -1685,6 +1757,8 @@ public final PullResponse pull(String subscription, boolean returnImmediately, i
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   PullRequest request =
    *       PullRequest.newBuilder()
@@ -1785,6 +1859,8 @@ final PullResponse pull(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   PullRequest request =
    *       PullRequest.newBuilder()
@@ -1814,6 +1890,8 @@ public final UnaryCallable pullCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   BidiStream bidiStream =
    *       subscriptionAdminClient.streamingPullCallable().call();
@@ -1852,6 +1930,8 @@ public final UnaryCallable pullCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
    *   PushConfig pushConfig = PushConfig.newBuilder().build();
@@ -1888,6 +1968,8 @@ public final void modifyPushConfig(SubscriptionName subscription, PushConfig pus
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
    *   PushConfig pushConfig = PushConfig.newBuilder().build();
@@ -1924,6 +2006,8 @@ public final void modifyPushConfig(String subscription, PushConfig pushConfig) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ModifyPushConfigRequest request =
    *       ModifyPushConfigRequest.newBuilder()
@@ -1991,6 +2075,8 @@ public final void modifyPushConfig(ProjectSubscriptionName subscription, PushCon
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ModifyPushConfigRequest request =
    *       ModifyPushConfigRequest.newBuilder()
@@ -2018,6 +2104,8 @@ public final UnaryCallable modifyPushConfigCalla
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SnapshotName snapshot = SnapshotName.of("[PROJECT]", "[SNAPSHOT]");
    *   Snapshot response = subscriptionAdminClient.getSnapshot(snapshot);
@@ -2046,6 +2134,8 @@ public final Snapshot getSnapshot(SnapshotName snapshot) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String snapshot = SnapshotName.of("[PROJECT]", "[SNAPSHOT]").toString();
    *   Snapshot response = subscriptionAdminClient.getSnapshot(snapshot);
@@ -2071,6 +2161,8 @@ public final Snapshot getSnapshot(String snapshot) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   GetSnapshotRequest request =
    *       GetSnapshotRequest.newBuilder()
@@ -2127,6 +2219,8 @@ public final Snapshot getSnapshot(ProjectSnapshotName snapshot) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   GetSnapshotRequest request =
    *       GetSnapshotRequest.newBuilder()
@@ -2153,6 +2247,8 @@ public final UnaryCallable getSnapshotCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ProjectName project = ProjectName.of("[PROJECT]");
    *   for (Snapshot element : subscriptionAdminClient.listSnapshots(project).iterateAll()) {
@@ -2183,6 +2279,8 @@ public final ListSnapshotsPagedResponse listSnapshots(ProjectName project) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String project = ProjectName.of("[PROJECT]").toString();
    *   for (Snapshot element : subscriptionAdminClient.listSnapshots(project).iterateAll()) {
@@ -2210,6 +2308,8 @@ public final ListSnapshotsPagedResponse listSnapshots(String project) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ListSnapshotsRequest request =
    *       ListSnapshotsRequest.newBuilder()
@@ -2240,6 +2340,8 @@ public final ListSnapshotsPagedResponse listSnapshots(ListSnapshotsRequest reque
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ListSnapshotsRequest request =
    *       ListSnapshotsRequest.newBuilder()
@@ -2271,6 +2373,8 @@ public final ListSnapshotsPagedResponse listSnapshots(ListSnapshotsRequest reque
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   ListSnapshotsRequest request =
    *       ListSnapshotsRequest.newBuilder()
@@ -2316,6 +2420,8 @@ public final UnaryCallable listSnap
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SnapshotName name = SnapshotName.of("[PROJECT]", "[SNAPSHOT]");
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
@@ -2364,6 +2470,8 @@ public final Snapshot createSnapshot(SnapshotName name, SubscriptionName subscri
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SnapshotName name = SnapshotName.of("[PROJECT]", "[SNAPSHOT]");
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
@@ -2412,6 +2520,8 @@ public final Snapshot createSnapshot(SnapshotName name, String subscription) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String name = SnapshotName.of("[PROJECT]", "[SNAPSHOT]").toString();
    *   SubscriptionName subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
@@ -2460,6 +2570,8 @@ public final Snapshot createSnapshot(String name, SubscriptionName subscription)
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String name = SnapshotName.of("[PROJECT]", "[SNAPSHOT]").toString();
    *   String subscription = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]").toString();
@@ -2505,6 +2617,8 @@ public final Snapshot createSnapshot(String name, String subscription) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   CreateSnapshotRequest request =
    *       CreateSnapshotRequest.newBuilder()
@@ -2692,6 +2806,8 @@ public final Snapshot createSnapshot(ProjectSnapshotName name, String subscripti
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   CreateSnapshotRequest request =
    *       CreateSnapshotRequest.newBuilder()
@@ -2720,6 +2836,8 @@ public final UnaryCallable createSnapshotCallab
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   UpdateSnapshotRequest request =
    *       UpdateSnapshotRequest.newBuilder()
@@ -2747,6 +2865,8 @@ public final Snapshot updateSnapshot(UpdateSnapshotRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   UpdateSnapshotRequest request =
    *       UpdateSnapshotRequest.newBuilder()
@@ -2777,6 +2897,8 @@ public final UnaryCallable updateSnapshotCallab
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SnapshotName snapshot = SnapshotName.of("[PROJECT]", "[SNAPSHOT]");
    *   subscriptionAdminClient.deleteSnapshot(snapshot);
@@ -2808,6 +2930,8 @@ public final void deleteSnapshot(SnapshotName snapshot) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   String snapshot = SnapshotName.of("[PROJECT]", "[SNAPSHOT]").toString();
    *   subscriptionAdminClient.deleteSnapshot(snapshot);
@@ -2837,6 +2961,8 @@ public final void deleteSnapshot(String snapshot) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   DeleteSnapshotRequest request =
    *       DeleteSnapshotRequest.newBuilder()
@@ -2899,6 +3025,8 @@ public final void deleteSnapshot(ProjectSnapshotName snapshot) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   DeleteSnapshotRequest request =
    *       DeleteSnapshotRequest.newBuilder()
@@ -2927,6 +3055,8 @@ public final UnaryCallable deleteSnapshotCallable(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SeekRequest request =
    *       SeekRequest.newBuilder()
@@ -2955,6 +3085,8 @@ public final SeekResponse seek(SeekRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SeekRequest request =
    *       SeekRequest.newBuilder()
@@ -2979,6 +3111,8 @@ public final UnaryCallable seekCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SetIamPolicyRequest request =
    *       SetIamPolicyRequest.newBuilder()
@@ -3037,6 +3171,8 @@ public final Policy setIamPolicy(String resource, Policy policy) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   SetIamPolicyRequest request =
    *       SetIamPolicyRequest.newBuilder()
@@ -3062,6 +3198,8 @@ public final UnaryCallable setIamPolicyCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   GetIamPolicyRequest request =
    *       GetIamPolicyRequest.newBuilder()
@@ -3112,6 +3250,8 @@ public final Policy getIamPolicy(String resource) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   GetIamPolicyRequest request =
    *       GetIamPolicyRequest.newBuilder()
@@ -3140,6 +3280,8 @@ public final UnaryCallable getIamPolicyCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   TestIamPermissionsRequest request =
    *       TestIamPermissionsRequest.newBuilder()
@@ -3207,6 +3349,8 @@ public final TestIamPermissionsResponse testIamPermissions(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
    *   TestIamPermissionsRequest request =
    *       TestIamPermissionsRequest.newBuilder()
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminSettings.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminSettings.java
index a90bfd141..c658184ab 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminSettings.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/SubscriptionAdminSettings.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -82,6 +82,8 @@
  * 

For example, to set the total timeout of createSubscription to 30 seconds: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SubscriptionAdminSettings.Builder subscriptionAdminSettingsBuilder =
  *     SubscriptionAdminSettings.newBuilder();
  * subscriptionAdminSettingsBuilder
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java
index 1d3da0057..0aa458c06 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminClient.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -66,6 +66,8 @@
  * calls that map to API methods. Sample code to get started:
  *
  * 
{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
  *   TopicName name = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
  *   Topic response = topicAdminClient.createTopic(name);
@@ -101,6 +103,8 @@
  * 

To customize credentials: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * TopicAdminSettings topicAdminSettings =
  *     TopicAdminSettings.newBuilder()
  *         .setCredentialsProvider(FixedCredentialsProvider.create(myCredentials))
@@ -111,6 +115,8 @@
  * 

To customize the endpoint: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * TopicAdminSettings topicAdminSettings =
  *     TopicAdminSettings.newBuilder().setEndpoint(myEndpoint).build();
  * TopicAdminClient topicAdminClient = TopicAdminClient.create(topicAdminSettings);
@@ -177,6 +183,8 @@ public PublisherStub getStub() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TopicName name = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
    *   Topic response = topicAdminClient.createTopic(name);
@@ -203,6 +211,8 @@ public final Topic createTopic(TopicName name) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   String name = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
    *   Topic response = topicAdminClient.createTopic(name);
@@ -258,6 +268,8 @@ public final Topic createTopic(ProjectTopicName name) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   Topic request =
    *       Topic.newBuilder()
@@ -288,6 +300,8 @@ public final Topic createTopic(Topic request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   Topic request =
    *       Topic.newBuilder()
@@ -316,6 +330,8 @@ public final UnaryCallable createTopicCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   UpdateTopicRequest request =
    *       UpdateTopicRequest.newBuilder()
@@ -340,6 +356,8 @@ public final Topic updateTopic(UpdateTopicRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   UpdateTopicRequest request =
    *       UpdateTopicRequest.newBuilder()
@@ -363,6 +381,8 @@ public final UnaryCallable updateTopicCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
    *   List messages = new ArrayList<>();
@@ -391,6 +411,8 @@ public final PublishResponse publish(TopicName topic, List messag
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   String topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
    *   List messages = new ArrayList<>();
@@ -416,6 +438,8 @@ public final PublishResponse publish(String topic, List messages)
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   PublishRequest request =
    *       PublishRequest.newBuilder()
@@ -440,6 +464,8 @@ public final PublishResponse publish(PublishRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   PublishRequest request =
    *       PublishRequest.newBuilder()
@@ -463,6 +489,8 @@ public final UnaryCallable publishCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
    *   Topic response = topicAdminClient.getTopic(topic);
@@ -486,6 +514,8 @@ public final Topic getTopic(TopicName topic) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   String topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
    *   Topic response = topicAdminClient.getTopic(topic);
@@ -533,6 +563,8 @@ public final Topic getTopic(ProjectTopicName topic) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   GetTopicRequest request =
    *       GetTopicRequest.newBuilder()
@@ -556,6 +588,8 @@ public final Topic getTopic(GetTopicRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   GetTopicRequest request =
    *       GetTopicRequest.newBuilder()
@@ -578,6 +612,8 @@ public final UnaryCallable getTopicCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ProjectName project = ProjectName.of("[PROJECT]");
    *   for (Topic element : topicAdminClient.listTopics(project).iterateAll()) {
@@ -605,6 +641,8 @@ public final ListTopicsPagedResponse listTopics(ProjectName project) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   String project = ProjectName.of("[PROJECT]").toString();
    *   for (Topic element : topicAdminClient.listTopics(project).iterateAll()) {
@@ -629,6 +667,8 @@ public final ListTopicsPagedResponse listTopics(String project) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicsRequest request =
    *       ListTopicsRequest.newBuilder()
@@ -656,6 +696,8 @@ public final ListTopicsPagedResponse listTopics(ListTopicsRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicsRequest request =
    *       ListTopicsRequest.newBuilder()
@@ -682,6 +724,8 @@ public final UnaryCallable listTopic
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicsRequest request =
    *       ListTopicsRequest.newBuilder()
@@ -715,6 +759,8 @@ public final UnaryCallable listTopicsCall
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
    *   for (String element : topicAdminClient.listTopicSubscriptions(topic).iterateAll()) {
@@ -742,6 +788,8 @@ public final ListTopicSubscriptionsPagedResponse listTopicSubscriptions(TopicNam
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   String topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
    *   for (String element : topicAdminClient.listTopicSubscriptions(topic).iterateAll()) {
@@ -796,6 +844,8 @@ public final ListTopicSubscriptionsPagedResponse listTopicSubscriptions(ProjectT
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicSubscriptionsRequest request =
    *       ListTopicSubscriptionsRequest.newBuilder()
@@ -824,6 +874,8 @@ public final ListTopicSubscriptionsPagedResponse listTopicSubscriptions(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicSubscriptionsRequest request =
    *       ListTopicSubscriptionsRequest.newBuilder()
@@ -852,6 +904,8 @@ public final ListTopicSubscriptionsPagedResponse listTopicSubscriptions(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicSubscriptionsRequest request =
    *       ListTopicSubscriptionsRequest.newBuilder()
@@ -890,6 +944,8 @@ public final ListTopicSubscriptionsPagedResponse listTopicSubscriptions(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
    *   for (String element : topicAdminClient.listTopicSnapshots(topic).iterateAll()) {
@@ -920,6 +976,8 @@ public final ListTopicSnapshotsPagedResponse listTopicSnapshots(TopicName topic)
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   String topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
    *   for (String element : topicAdminClient.listTopicSnapshots(topic).iterateAll()) {
@@ -948,6 +1006,8 @@ public final ListTopicSnapshotsPagedResponse listTopicSnapshots(String topic) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicSnapshotsRequest request =
    *       ListTopicSnapshotsRequest.newBuilder()
@@ -979,6 +1039,8 @@ public final ListTopicSnapshotsPagedResponse listTopicSnapshots(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicSnapshotsRequest request =
    *       ListTopicSnapshotsRequest.newBuilder()
@@ -1010,6 +1072,8 @@ public final ListTopicSnapshotsPagedResponse listTopicSnapshots(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   ListTopicSnapshotsRequest request =
    *       ListTopicSnapshotsRequest.newBuilder()
@@ -1048,6 +1112,8 @@ public final ListTopicSnapshotsPagedResponse listTopicSnapshots(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
    *   topicAdminClient.deleteTopic(topic);
@@ -1074,6 +1140,8 @@ public final void deleteTopic(TopicName topic) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   String topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]").toString();
    *   topicAdminClient.deleteTopic(topic);
@@ -1127,6 +1195,8 @@ public final void deleteTopic(ProjectTopicName topic) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   DeleteTopicRequest request =
    *       DeleteTopicRequest.newBuilder()
@@ -1153,6 +1223,8 @@ public final void deleteTopic(DeleteTopicRequest request) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   DeleteTopicRequest request =
    *       DeleteTopicRequest.newBuilder()
@@ -1177,6 +1249,8 @@ public final UnaryCallable deleteTopicCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   DetachSubscriptionRequest request =
    *       DetachSubscriptionRequest.newBuilder()
@@ -1202,6 +1276,8 @@ public final DetachSubscriptionResponse detachSubscription(DetachSubscriptionReq
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   DetachSubscriptionRequest request =
    *       DetachSubscriptionRequest.newBuilder()
@@ -1228,6 +1304,8 @@ public final DetachSubscriptionResponse detachSubscription(DetachSubscriptionReq
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   SetIamPolicyRequest request =
    *       SetIamPolicyRequest.newBuilder()
@@ -1286,6 +1364,8 @@ public final Policy setIamPolicy(String resource, Policy policy) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   SetIamPolicyRequest request =
    *       SetIamPolicyRequest.newBuilder()
@@ -1311,6 +1391,8 @@ public final UnaryCallable setIamPolicyCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   GetIamPolicyRequest request =
    *       GetIamPolicyRequest.newBuilder()
@@ -1361,6 +1443,8 @@ public final Policy getIamPolicy(String resource) {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   GetIamPolicyRequest request =
    *       GetIamPolicyRequest.newBuilder()
@@ -1389,6 +1473,8 @@ public final UnaryCallable getIamPolicyCallable() {
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TestIamPermissionsRequest request =
    *       TestIamPermissionsRequest.newBuilder()
@@ -1456,6 +1542,8 @@ public final TestIamPermissionsResponse testIamPermissions(
    * 

Sample code: * *

{@code
+   * // This snippet has been automatically generated for illustrative purposes only.
+   * // It may require modifications to work in your environment.
    * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
    *   TestIamPermissionsRequest request =
    *       TestIamPermissionsRequest.newBuilder()
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminSettings.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminSettings.java
index 41ebfd68d..d0efac54f 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminSettings.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/TopicAdminSettings.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -75,6 +75,8 @@
  * 

For example, to set the total timeout of createTopic to 30 seconds: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * TopicAdminSettings.Builder topicAdminSettingsBuilder = TopicAdminSettings.newBuilder();
  * topicAdminSettingsBuilder
  *     .createTopicSettings()
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/package-info.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/package-info.java
index 128d54247..6e17761f9 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/package-info.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/package-info.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -27,6 +27,8 @@
  * 

Sample for TopicAdminClient: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
  *   TopicName name = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
  *   Topic response = topicAdminClient.createTopic(name);
@@ -42,6 +44,8 @@
  * 

Sample for SubscriptionAdminClient: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
  *   SubscriptionName name = SubscriptionName.of("[PROJECT]", "[SUBSCRIPTION]");
  *   TopicName topic = TopicName.ofProjectTopicName("[PROJECT]", "[TOPIC]");
@@ -59,6 +63,8 @@
  * 

Sample for SchemaServiceClient: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * try (SchemaServiceClient schemaServiceClient = SchemaServiceClient.create()) {
  *   ProjectName parent = ProjectName.of("[PROJECT]");
  *   Schema schema = Schema.newBuilder().build();
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherCallableFactory.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherCallableFactory.java
index 35e7f539a..140ba0474 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherCallableFactory.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherCallableFactory.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherStub.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherStub.java
index 2b2e725c1..53607aafe 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherStub.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcPublisherStub.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceCallableFactory.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceCallableFactory.java
index b1e3e27cc..25bb84500 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceCallableFactory.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceCallableFactory.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceStub.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceStub.java
index e5cd65fb6..4d06023b7 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceStub.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSchemaServiceStub.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberCallableFactory.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberCallableFactory.java
index 97d867601..555d231b3 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberCallableFactory.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberCallableFactory.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberStub.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberStub.java
index 523c0376a..e199d9ae3 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberStub.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/GrpcSubscriberStub.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStub.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStub.java
index 9b90ca986..c26fe479c 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStub.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStub.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStubSettings.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStubSettings.java
index 166775ec3..aaeb343d3 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStubSettings.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/PublisherStubSettings.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -99,6 +99,8 @@
  * 

For example, to set the total timeout of createTopic to 30 seconds: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * PublisherStubSettings.Builder topicAdminSettingsBuilder = PublisherStubSettings.newBuilder();
  * topicAdminSettingsBuilder
  *     .createTopicSettings()
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStub.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStub.java
index b194653b3..47aac9109 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStub.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStub.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStubSettings.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStubSettings.java
index fb8a87653..a7d8fc4ae 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStubSettings.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SchemaServiceStubSettings.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -82,6 +82,8 @@
  * 

For example, to set the total timeout of createSchema to 30 seconds: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SchemaServiceStubSettings.Builder schemaServiceSettingsBuilder =
  *     SchemaServiceStubSettings.newBuilder();
  * schemaServiceSettingsBuilder
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStub.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStub.java
index 7233e2008..8eeab5261 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStub.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStub.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStubSettings.java b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStubSettings.java
index 32733fbf9..11a87f726 100644
--- a/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStubSettings.java
+++ b/google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/stub/SubscriberStubSettings.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -97,6 +97,8 @@
  * 

For example, to set the total timeout of createSubscription to 30 seconds: * *

{@code
+ * // This snippet has been automatically generated for illustrative purposes only.
+ * // It may require modifications to work in your environment.
  * SubscriberStubSettings.Builder subscriptionAdminSettingsBuilder =
  *     SubscriberStubSettings.newBuilder();
  * subscriptionAdminSettingsBuilder
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicy.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicy.java
index e7583f2d9..85c3e07a0 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicy.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicy.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicyImpl.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicyImpl.java
index e4528ae53..089b0b380 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicyImpl.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockIAMPolicyImpl.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisher.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisher.java
index 78513630a..31d2b2b09 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisher.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisher.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisherImpl.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisherImpl.java
index 920917858..53f88cfe6 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisherImpl.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockPublisherImpl.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaService.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaService.java
index a7fc32c58..4ebffafed 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaService.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaService.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaServiceImpl.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaServiceImpl.java
index 5b908db48..6124aab4d 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaServiceImpl.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSchemaServiceImpl.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriber.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriber.java
index 8d3f29fe0..2615733f7 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriber.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriber.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriberImpl.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriberImpl.java
index a181138d9..9747d2ca0 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriberImpl.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/MockSubscriberImpl.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java
index 864f9f013..92673b9d9 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SchemaServiceClientTest.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java
index dbb87a03e..d3402991a 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/SubscriptionAdminClientTest.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java
index a3c4e4245..0d50563fe 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/TopicAdminClientTest.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ProjectName.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ProjectName.java
index 4050e3e1a..9526c02ab 100644
--- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ProjectName.java
+++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/ProjectName.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaName.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaName.java
index af7c5feab..c4b5935cd 100644
--- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaName.java
+++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SchemaName.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotName.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotName.java
index fe26eefe0..d415eefb4 100644
--- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotName.java
+++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SnapshotName.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionName.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionName.java
index 0eed7889a..6433bd9f9 100644
--- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionName.java
+++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/SubscriptionName.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
diff --git a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicName.java b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicName.java
index dcb2fa83b..4d16d44af 100644
--- a/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicName.java
+++ b/proto-google-cloud-pubsub-v1/src/main/java/com/google/pubsub/v1/TopicName.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2021 Google LLC
+ * Copyright 2022 Google LLC
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.

From 53ebb0ba84b9973260c82cd5fd0c2a834ded4ea2 Mon Sep 17 00:00:00 2001
From: WhiteSource Renovate 
Date: Fri, 6 May 2022 00:30:20 +0200
Subject: [PATCH 50/52] build(deps): update dependency
 org.apache.maven.plugins:maven-project-info-reports-plugin to v3.3.0 (#1104)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

[![WhiteSource Renovate](https://app.renovatebot.com/images/banner.svg)](https://renovatebot.com)

This PR contains the following updates:

| Package | Change | Age | Adoption | Passing | Confidence |
|---|---|---|---|---|---|
| [org.apache.maven.plugins:maven-project-info-reports-plugin](https://maven.apache.org/plugins/) ([source](https://togithub.com/apache/maven-project-info-reports-plugin)) | `3.2.2` -> `3.3.0` | [![age](https://badges.renovateapi.com/packages/maven/org.apache.maven.plugins:maven-project-info-reports-plugin/3.3.0/age-slim)](https://docs.renovatebot.com/merge-confidence/) | [![adoption](https://badges.renovateapi.com/packages/maven/org.apache.maven.plugins:maven-project-info-reports-plugin/3.3.0/adoption-slim)](https://docs.renovatebot.com/merge-confidence/) | [![passing](https://badges.renovateapi.com/packages/maven/org.apache.maven.plugins:maven-project-info-reports-plugin/3.3.0/compatibility-slim/3.2.2)](https://docs.renovatebot.com/merge-confidence/) | [![confidence](https://badges.renovateapi.com/packages/maven/org.apache.maven.plugins:maven-project-info-reports-plugin/3.3.0/confidence-slim/3.2.2)](https://docs.renovatebot.com/merge-confidence/) |

---

### Configuration

📅 **Schedule**: At any time (no schedule defined).

🚦 **Automerge**: Disabled by config. Please merge this manually once you are satisfied.

â™» **Rebasing**: Whenever PR becomes conflicted, or you tick the rebase/retry checkbox.

🔕 **Ignore**: Close this PR and you won't be reminded about this update again.

---

 - [ ] If you want to rebase/retry this PR, click this checkbox.

---

This PR has been generated by [WhiteSource Renovate](https://renovate.whitesourcesoftware.com). View repository job log [here](https://app.renovatebot.com/dashboard#github/googleapis/java-pubsub).
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index 716e6366e..24c8f2329 100644
--- a/pom.xml
+++ b/pom.xml
@@ -145,7 +145,7 @@
       
         org.apache.maven.plugins
         maven-project-info-reports-plugin
-        3.2.2
+        3.3.0
         
           
             

From 1fb553c70ec81c4571ab895d64382bf126ecd343 Mon Sep 17 00:00:00 2001
From: Rajanya Dhar 
Date: Tue, 8 Feb 2022 12:09:37 -0500
Subject: [PATCH 51/52] Formatting

---
 .../test/java/com/google/cloud/pubsub/it/ITPubSubTest.java   | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java
index 290b9927d..39b35c341 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java
@@ -435,7 +435,10 @@ public void failed(Subscriber.State from, Throwable failure) {
         MoreExecutors.directExecutor());
     subscriber.startAsync();
 
-    Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build();
+    Publisher publisher =
+        Publisher.newBuilder(topicName)
+            .setEnableCompression(true)
+            .build();
 
     String msg1 = generateMessage("msg1", 1000);
     String msg2 = generateMessage("msg2", 1500);

From f83bcc78964cd32becf448345e6bcf3c8aaae65a Mon Sep 17 00:00:00 2001
From: Rajanya Dhar 
Date: Mon, 9 May 2022 11:00:53 -0400
Subject: [PATCH 52/52] Formatting

---
 .../test/java/com/google/cloud/pubsub/it/ITPubSubTest.java   | 5 +----
 1 file changed, 1 insertion(+), 4 deletions(-)

diff --git a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java
index 39b35c341..290b9927d 100644
--- a/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java
+++ b/google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/it/ITPubSubTest.java
@@ -435,10 +435,7 @@ public void failed(Subscriber.State from, Throwable failure) {
         MoreExecutors.directExecutor());
     subscriber.startAsync();
 
-    Publisher publisher =
-        Publisher.newBuilder(topicName)
-            .setEnableCompression(true)
-            .build();
+    Publisher publisher = Publisher.newBuilder(topicName).setEnableCompression(true).build();
 
     String msg1 = generateMessage("msg1", 1000);
     String msg2 = generateMessage("msg2", 1500);