Skip to content

Commit

Permalink
samples: add exactly once subscribe sample (#1029)
Browse files Browse the repository at this point in the history
* created exactly once subscription example

* updated example

* updated sample

* removed tag

* cleanup + add tests

* checkstyle

* formatting

* updated some comments, added try-catch

* added switch statement to handle ackResponses + minDuration option

* added (redundant) default for enum switch statement

* copy-paste issue

* changed wording in sample

* updated wording

* code review changes

* update comments

* Update samples/snippets/src/main/java/pubsub/SubscribeWithExactlyOnceConsumerWithResponseExample.java

Co-authored-by: Tianzi Cai <tianzi@google.com>

* update comments

* add a test subscription with exactly once delivery

* add imports

* remove topicId

* updated samples tests

* removed extra space

* handling null message

* combined a try-catch

* added new topic in sample test for exactly once delivery

* added cleanup

* fixed formatting

* 🦉 Updates from OwlBot post-processor

See https://github.com/googleapis/repo-automation-bots/blob/main/packages/owl-bot/README.md

Co-authored-by: Tianzi Cai <tianzi@google.com>
Co-authored-by: Owl Bot <gcf-owl-bot[bot]@users.noreply.github.com>
  • Loading branch information
3 people committed Mar 15, 2022
1 parent 0b7d19f commit fe583cb
Show file tree
Hide file tree
Showing 3 changed files with 149 additions and 2 deletions.
1 change: 1 addition & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -285,6 +285,7 @@ Samples are in the [`samples/`](https://github.com/googleapis/java-pubsub/tree/m
| Subscribe With Concurrency Control Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/SubscribeWithConcurrencyControlExample.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/SubscribeWithConcurrencyControlExample.java) |
| Subscribe With Custom Attributes Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/SubscribeWithCustomAttributesExample.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/SubscribeWithCustomAttributesExample.java) |
| Subscribe With Error Listener Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/SubscribeWithErrorListenerExample.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/SubscribeWithErrorListenerExample.java) |
| Subscribe With Exactly Once Consumer With Response Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/SubscribeWithExactlyOnceConsumerWithResponseExample.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/SubscribeWithExactlyOnceConsumerWithResponseExample.java) |
| Subscribe With Flow Control Settings Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/SubscribeWithFlowControlSettingsExample.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/SubscribeWithFlowControlSettingsExample.java) |
| Subscribe With Proto Schema Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/SubscribeWithProtoSchemaExample.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/SubscribeWithProtoSchemaExample.java) |
| Test Subscription Permissions Example | [source code](https://github.com/googleapis/java-pubsub/blob/main/samples/snippets/src/main/java/pubsub/TestSubscriptionPermissionsExample.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/TestSubscriptionPermissionsExample.java) |
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/*
* 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_subscriber_exactly_once]

import com.google.cloud.pubsub.v1.AckReplyConsumerWithResponse;
import com.google.cloud.pubsub.v1.AckResponse;
import com.google.cloud.pubsub.v1.MessageReceiverWithAckResponse;
import com.google.cloud.pubsub.v1.Subscriber;
import com.google.pubsub.v1.ProjectSubscriptionName;
import com.google.pubsub.v1.PubsubMessage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;

public class SubscribeWithExactlyOnceConsumerWithResponseExample {
public static void main(String... args) throws Exception {
// TODO(developer): Replace these variables before running the sample.
String projectId = "your-project-id";
String subscriptionId = "your-subscription-id";

subscribeWithExactlyOnceConsumerWithResponseExample(projectId, subscriptionId);
}

public static void subscribeWithExactlyOnceConsumerWithResponseExample(
String projectId, String subscriptionId) {
ProjectSubscriptionName subscriptionName =
ProjectSubscriptionName.of(projectId, subscriptionId);

// Instantiate an asynchronous message receiver using `AckReplyConsumerWithResponse`
// instead of `AckReplyConsumer` to get a future that tracks the result of the ack call.
// When exactly once delivery is enabled on the subscription, the message is guaranteed
// to not be delivered again if the ack future succeeds.
MessageReceiverWithAckResponse receiverWithResponse =
(PubsubMessage message, AckReplyConsumerWithResponse consumerWithResponse) -> {
try {
// Handle incoming message, then ack the message, and receive an ack response.
System.out.println("Message received: " + message.getData().toStringUtf8());
Future<AckResponse> ackResponseFuture = consumerWithResponse.ack();

// Retrieve the completed future for the ack response from the server.
AckResponse ackResponse = ackResponseFuture.get();

switch (ackResponse) {
case SUCCESSFUL:
// Success code means that this MessageID will not be delivered again.
System.out.println("Message successfully acked: " + message.getMessageId());
break;
case INVALID:
System.out.println(
"Message failed to ack with a response of Invalid. Id: "
+ message.getMessageId());
break;
case PERMISSION_DENIED:
System.out.println(
"Message failed to ack with a response of Permission Denied. Id: "
+ message.getMessageId());
break;
case FAILED_PRECONDITION:
System.out.println(
"Message failed to ack with a response of Failed Precondition. Id: "
+ message.getMessageId());
break;
case OTHER:
System.out.println(
"Message failed to ack with a response of Other. Id: "
+ message.getMessageId());
break;
default:
break;
}
} catch (InterruptedException | ExecutionException e) {
System.out.println(
"MessageId: " + message.getMessageId() + " failed when retrieving future");
} catch (Throwable t) {
System.out.println("Throwable caught" + t.getMessage());
}
};

Subscriber subscriber = null;
try {
subscriber = Subscriber.newBuilder(subscriptionName, receiverWithResponse).build();
// Start the subscriber.
subscriber.startAsync().awaitRunning();
System.out.printf("Listening for messages on %s:\n", subscriptionName.toString());
// Allow the subscriber to run for 30s unless an unrecoverable error occurs.
subscriber.awaitTerminated(30, TimeUnit.SECONDS);
} catch (TimeoutException timeoutException) {
// Shut down the subscriber after 30s. Stop receiving messages.
subscriber.stopAsync();
}
}
}
// [END pubsub_subscriber_exactly_once]
40 changes: 38 additions & 2 deletions samples/snippets/src/test/java/pubsub/SubscriberIT.java
Original file line number Diff line number Diff line change
Expand Up @@ -52,19 +52,30 @@ public class SubscriberIT {
private static final String projectId = System.getenv("GOOGLE_CLOUD_PROJECT");
private static final String _suffix = UUID.randomUUID().toString();
private static final String topicId = "subscriber-test-topic-" + _suffix;
private static final String topicIdEod = "subscriber-test-topic-eod" + _suffix;
private static final String subscriptionId = "subscriber-test-subscription-" + _suffix;
// For a subscription with exactly once delivery enabled.
private static final String subscriptionEodId = "subscriber-test-subscription-eod" + _suffix;
private static final TopicName topicName = TopicName.of(projectId, topicId);
private static final TopicName topicNameEod = TopicName.of(projectId, topicIdEod);
private static final ProjectSubscriptionName subscriptionName =
ProjectSubscriptionName.of(projectId, subscriptionId);
private static final ProjectSubscriptionName subscriptionEodName =
ProjectSubscriptionName.of(projectId, subscriptionEodId);

private static void requireEnvVar(String varName) {
assertNotNull(
"Environment variable " + varName + " is required to perform these tests.",
System.getenv(varName));
}

private static List<String> publishSomeMessages(Integer numOfMessages) throws Exception {
return publishSomeMessages(numOfMessages, topicId);
}

// Helper function to publish some messages.
private static void publishSomeMessages(Integer numOfMessages) throws Exception {
private static List<String> publishSomeMessages(Integer numOfMessages, String topicId)
throws Exception {
ProjectTopicName topicName = ProjectTopicName.of(projectId, topicId);
Publisher publisher = Publisher.newBuilder(topicName).build();
List<ApiFuture<String>> messageIdFutures = new ArrayList<>();
Expand All @@ -78,7 +89,7 @@ private static void publishSomeMessages(Integer numOfMessages) throws Exception
ApiFuture<String> messageIdFuture = publisher.publish(pubsubMessage);
messageIdFutures.add(messageIdFuture);
}
ApiFutures.allAsList(messageIdFutures).get();
return ApiFutures.allAsList(messageIdFutures).get();
}

// Helper function to retry synchronous pull attempts until all outstanding messages are received.
Expand Down Expand Up @@ -123,6 +134,9 @@ public void setUp() throws Exception {
try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
Topic topic = Topic.newBuilder().setName(topicName.toString()).build();
topicAdminClient.createTopic(topic);

Topic topicEod = Topic.newBuilder().setName(topicNameEod.toString()).build();
topicAdminClient.createTopic(topicEod);
}

try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
Expand All @@ -132,17 +146,28 @@ public void setUp() throws Exception {
.setTopic(topicName.toString())
.build();
subscriptionAdminClient.createSubscription(subscription);

Subscription subscriptionEod =
Subscription.newBuilder()
.setName(subscriptionEodName.toString())
.setTopic(topicNameEod.toString())
// Enable exactly once delivery in the subscription.
.setEnableExactlyOnceDelivery(true)
.build();
subscriptionAdminClient.createSubscription(subscriptionEod);
}
}

@After
public void tearDown() throws Exception {
try (SubscriptionAdminClient subscriptionAdminClient = SubscriptionAdminClient.create()) {
subscriptionAdminClient.deleteSubscription(subscriptionName.toString());
subscriptionAdminClient.deleteSubscription(subscriptionEodName.toString());
}

try (TopicAdminClient topicAdminClient = TopicAdminClient.create()) {
topicAdminClient.deleteTopic(topicName.toString());
topicAdminClient.deleteTopic(topicNameEod.toString());
}

System.setOut(null);
Expand Down Expand Up @@ -204,4 +229,15 @@ public void testSubscriber() throws Exception {
SubscribeSyncWithLeaseExample.subscribeSyncWithLeaseExample(
projectId, subscriptionId, 10));
}

@Test
public void testSubscriberExactlyOnceDelivery() throws Exception {
List<String> messageIds = publishSomeMessages(10, topicIdEod);
bout.reset();
SubscribeWithExactlyOnceConsumerWithResponseExample
.subscribeWithExactlyOnceConsumerWithResponseExample(projectId, subscriptionEodId);
for (String messageId : messageIds) {
assertThat(bout.toString()).contains("Message successfully acked: " + messageId);
}
}
}

0 comments on commit fe583cb

Please sign in to comment.