Skip to content

Commit 00070b7

Browse files
authored
fix: Add retries for ack and modack operations that don't return with a metadata map (#2385)
* fix: Add retries for ack and modack operations that don't return with a metadata map * fix: Remove messages from pending requests if the ack/modack failure does not have a metadata map * fix: Forget ack IDs when ack/modacks fail with non-retryable errors
1 parent 4119cc0 commit 00070b7

File tree

5 files changed

+174
-55
lines changed

5 files changed

+174
-55
lines changed

google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/MessageDispatcher.java

+15
Original file line numberDiff line numberDiff line change
@@ -476,6 +476,21 @@ void notifyAckFailed(AckRequestData ackRequestData) {
476476
synchronized (outstandingReceipts) {
477477
outstandingReceipts.remove(ackRequestData.getAckId());
478478
}
479+
480+
// When notifying that an ack/modack has failed, due to a non-retryable error,
481+
// we attempt to remove the message from the pending messages and release it from the flow
482+
// controller so that we no longer attempt to extend the message's ack deadline.
483+
if (pendingMessages.remove(ackRequestData.getAckId()) == null) {
484+
/*
485+
* We're forgetting the message for the second time. This may occur on modacks because the message passed
486+
* its total expiration and was forgotten and then the user finishes working on the message
487+
* which forgets the message again. Additionally, when a failed ack occurs, we will have already forgotten
488+
* the message, so we don't need to here. Turns the second forget into a no-op so we don't free twice.
489+
*/
490+
return;
491+
}
492+
flowController.release(1, ackRequestData.getMessageWrapper().getSerializedSize());
493+
messagesWaiter.incrementPendingCount(-1);
479494
}
480495

481496
private void processBatch(List<OutstandingMessage> batch) {

google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/PubsubMessageWrapper.java

+4
Original file line numberDiff line numberDiff line change
@@ -133,6 +133,10 @@ int getDataSize() {
133133
return message.getData().size();
134134
}
135135

136+
int getSerializedSize() {
137+
return message.getSerializedSize();
138+
}
139+
136140
String getOrderingKey() {
137141
return message.getOrderingKey();
138142
}

google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/StatusUtil.java

+20
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,26 @@ private StatusUtil() {
2525
// Static class, not instantiable.
2626
}
2727

28+
static AckResponse getFailedAckResponse(Throwable t) {
29+
if (!(t instanceof ApiException)) {
30+
return AckResponse.OTHER;
31+
}
32+
33+
ApiException apiException = (ApiException) t;
34+
AckResponse failedAckResponse;
35+
switch (apiException.getStatusCode().getCode()) {
36+
case FAILED_PRECONDITION:
37+
failedAckResponse = AckResponse.FAILED_PRECONDITION;
38+
break;
39+
case PERMISSION_DENIED:
40+
failedAckResponse = AckResponse.PERMISSION_DENIED;
41+
break;
42+
default:
43+
failedAckResponse = AckResponse.OTHER;
44+
}
45+
return failedAckResponse;
46+
}
47+
2848
static boolean isRetryable(Throwable error) {
2949
if (!(error instanceof ApiException)) {
3050
return true;

google-cloud-pubsub/src/main/java/com/google/cloud/pubsub/v1/StreamingSubscriberConnection.java

+73-55
Original file line numberDiff line numberDiff line change
@@ -408,21 +408,7 @@ private void setFailureFutureOutstandingMessages(Throwable t) {
408408
AckResponse ackResponse;
409409

410410
if (getExactlyOnceDeliveryEnabled()) {
411-
if (!(t instanceof ApiException)) {
412-
ackResponse = AckResponse.OTHER;
413-
}
414-
415-
ApiException apiException = (ApiException) t;
416-
switch (apiException.getStatusCode().getCode()) {
417-
case FAILED_PRECONDITION:
418-
ackResponse = AckResponse.FAILED_PRECONDITION;
419-
break;
420-
case PERMISSION_DENIED:
421-
ackResponse = AckResponse.PERMISSION_DENIED;
422-
break;
423-
default:
424-
ackResponse = AckResponse.OTHER;
425-
}
411+
ackResponse = StatusUtil.getFailedAckResponse(t);
426412
} else {
427413
// We should set success regardless if ExactlyOnceDelivery is not enabled
428414
ackResponse = AckResponse.SUCCESSFUL;
@@ -504,7 +490,7 @@ private void sendModackOperations(
504490
modackRequestData.getIsReceiptModack());
505491
ApiFutureCallback<Empty> callback =
506492
getCallback(
507-
modackRequestData.getAckRequestData(),
493+
ackRequestDataInRequestList,
508494
deadlineExtensionSeconds,
509495
true,
510496
currentBackoffMillis,
@@ -611,59 +597,91 @@ public void onFailure(Throwable t) {
611597
List<AckRequestData> ackRequestDataArrayRetryList = new ArrayList<>();
612598
try {
613599
Map<String, String> metadataMap = getMetadataMapFromThrowable(t);
614-
ackRequestDataList.forEach(
615-
ackRequestData -> {
616-
String ackId = ackRequestData.getAckId();
617-
if (metadataMap.containsKey(ackId)) {
618-
// An error occured
619-
String errorMessage = metadataMap.get(ackId);
620-
if (errorMessage.startsWith(TRANSIENT_FAILURE_METADATA_PREFIX)) {
621-
// Retry all "TRANSIENT_*" error messages - do not set message future
622-
logger.log(Level.INFO, "Transient error message, will resend", errorMessage);
623-
ackRequestDataArrayRetryList.add(ackRequestData);
624-
} else if (errorMessage.equals(PERMANENT_FAILURE_INVALID_ACK_ID_METADATA)) {
625-
// Permanent failure, send
626-
logger.log(
627-
Level.INFO,
628-
"Permanent error invalid ack id message, will not resend",
629-
errorMessage);
630-
ackRequestData.setResponse(AckResponse.INVALID, setResponseOnSuccess);
600+
if (metadataMap.isEmpty()) {
601+
String operation = isModack ? "ModifyAckDeadline" : "Acknowledge";
602+
if (!StatusUtil.isRetryable(t)) {
603+
logger.log(Level.WARNING, "Un-retryable error on " + operation, t);
604+
ackRequestDataList.forEach(
605+
ackRequestData -> {
606+
AckResponse failedAckResponse = StatusUtil.getFailedAckResponse(t);
607+
ackRequestData.setResponse(failedAckResponse, setResponseOnSuccess);
631608
messageDispatcher.notifyAckFailed(ackRequestData);
632609
tracer.addEndRpcEvent(
633610
ackRequestData.getMessageWrapper(),
634611
rpcSpanSampled,
635612
isModack,
636613
deadlineExtensionSeconds);
637614
tracer.setSubscriberSpanException(
638-
ackRequestData.getMessageWrapper(), t, "Invalid ack ID");
615+
ackRequestData.getMessageWrapper(), t, "Error with no metadata map");
616+
ackRequestData
617+
.getMessageWrapper()
618+
.setSubscriberSpanException(t, "Error with no metadata map");
619+
pendingRequests.remove(ackRequestData);
620+
});
621+
} else {
622+
logger.log(Level.INFO, "Retryable error on " + operation + ", will resend", t);
623+
ackRequestDataArrayRetryList.addAll(ackRequestDataList);
624+
ackRequestDataList.forEach(
625+
ackRequestData -> {
626+
pendingRequests.remove(ackRequestData);
627+
});
628+
}
629+
} else {
630+
ackRequestDataList.forEach(
631+
ackRequestData -> {
632+
String ackId = ackRequestData.getAckId();
633+
if (metadataMap.containsKey(ackId)) {
634+
// An error occured
635+
String errorMessage = metadataMap.get(ackId);
636+
if (errorMessage.startsWith(TRANSIENT_FAILURE_METADATA_PREFIX)) {
637+
// Retry all "TRANSIENT_*" error messages - do not set message future
638+
logger.log(Level.INFO, "Transient error message, will resend", errorMessage);
639+
ackRequestDataArrayRetryList.add(ackRequestData);
640+
} else if (errorMessage.equals(PERMANENT_FAILURE_INVALID_ACK_ID_METADATA)) {
641+
// Permanent failure
642+
logger.log(
643+
Level.INFO,
644+
"Permanent error invalid ack id message, will not resend",
645+
errorMessage);
646+
ackRequestData.setResponse(AckResponse.INVALID, setResponseOnSuccess);
647+
messageDispatcher.notifyAckFailed(ackRequestData);
648+
tracer.addEndRpcEvent(
649+
ackRequestData.getMessageWrapper(),
650+
rpcSpanSampled,
651+
isModack,
652+
deadlineExtensionSeconds);
653+
tracer.setSubscriberSpanException(
654+
ackRequestData.getMessageWrapper(), t, "Invalid ack ID");
655+
} else {
656+
logger.log(
657+
Level.INFO, "Unknown error message, will not resend", errorMessage);
658+
ackRequestData.setResponse(AckResponse.OTHER, setResponseOnSuccess);
659+
messageDispatcher.notifyAckFailed(ackRequestData);
660+
tracer.addEndRpcEvent(
661+
ackRequestData.getMessageWrapper(),
662+
rpcSpanSampled,
663+
isModack,
664+
deadlineExtensionSeconds);
665+
tracer.setSubscriberSpanException(
666+
ackRequestData.getMessageWrapper(), t, "Unknown error message");
667+
ackRequestData
668+
.getMessageWrapper()
669+
.setSubscriberSpanException(t, "Unknown error message");
670+
}
639671
} else {
640-
logger.log(Level.INFO, "Unknown error message, will not resend", errorMessage);
641-
ackRequestData.setResponse(AckResponse.OTHER, setResponseOnSuccess);
642-
messageDispatcher.notifyAckFailed(ackRequestData);
672+
ackRequestData.setResponse(AckResponse.SUCCESSFUL, setResponseOnSuccess);
673+
messageDispatcher.notifyAckSuccess(ackRequestData);
674+
tracer.endSubscriberSpan(ackRequestData.getMessageWrapper());
643675
tracer.addEndRpcEvent(
644676
ackRequestData.getMessageWrapper(),
645677
rpcSpanSampled,
646678
isModack,
647679
deadlineExtensionSeconds);
648-
tracer.setSubscriberSpanException(
649-
ackRequestData.getMessageWrapper(), t, "Unknown error message");
650-
ackRequestData
651-
.getMessageWrapper()
652-
.setSubscriberSpanException(t, "Unknown error message");
653680
}
654-
} else {
655-
ackRequestData.setResponse(AckResponse.SUCCESSFUL, setResponseOnSuccess);
656-
messageDispatcher.notifyAckSuccess(ackRequestData);
657-
tracer.endSubscriberSpan(ackRequestData.getMessageWrapper());
658-
tracer.addEndRpcEvent(
659-
ackRequestData.getMessageWrapper(),
660-
rpcSpanSampled,
661-
isModack,
662-
deadlineExtensionSeconds);
663-
}
664-
// Remove from our pending
665-
pendingRequests.remove(ackRequestData);
666-
});
681+
// Remove from our pending
682+
pendingRequests.remove(ackRequestData);
683+
});
684+
}
667685
} catch (InvalidProtocolBufferException e) {
668686
// If we fail to parse out the errorInfo, we should retry all
669687
logger.log(

google-cloud-pubsub/src/test/java/com/google/cloud/pubsub/v1/StreamingSubscriberConnectionTest.java

+62
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,16 @@
1717
package com.google.cloud.pubsub.v1;
1818

1919
import static org.junit.Assert.*;
20+
import static org.mockito.ArgumentMatchers.argThat;
2021
import static org.mockito.Mockito.*;
2122

2223
import com.google.api.core.ApiFutures;
2324
import com.google.api.core.SettableApiFuture;
2425
import com.google.api.gax.batching.FlowControlSettings;
2526
import com.google.api.gax.batching.FlowController;
2627
import com.google.api.gax.core.Distribution;
28+
import com.google.api.gax.grpc.GrpcStatusCode;
29+
import com.google.api.gax.rpc.ApiException;
2730
import com.google.api.gax.rpc.StatusCode;
2831
import com.google.cloud.pubsub.v1.stub.SubscriberStub;
2932
import com.google.common.collect.Lists;
@@ -32,6 +35,7 @@
3235
import com.google.pubsub.v1.ModifyAckDeadlineRequest;
3336
import com.google.rpc.ErrorInfo;
3437
import com.google.rpc.Status;
38+
import io.grpc.Status.Code;
3539
import io.grpc.StatusException;
3640
import io.grpc.protobuf.StatusProto;
3741
import java.time.Duration;
@@ -65,6 +69,8 @@ public class StreamingSubscriberConnectionTest {
6569
"MOCK-ACK-ID-TRANSIENT-FAILURE-SERVICE-UNAVAILABLE-THEN-SUCCESS";
6670
private static final String MOCK_ACK_ID_INVALID = "MOCK-ACK-ID-INVALID";
6771
private static final String MOCK_ACK_ID_OTHER = "MOCK-ACK-ID-OTHER";
72+
private static final String MOCK_ACK_ID_NO_METADATA_MAP_INTERNAL_ERROR_THEN_PERMISSION_DENIED =
73+
"MOCK-ACK-ID-NO-METADATA-MAP-INTERNAL-ERROR";
6874

6975
private static final String PERMANENT_FAILURE_INVALID_ACK_ID = "PERMANENT_FAILURE_INVALID_ACK_ID";
7076
private static final String TRANSIENT_FAILURE_UNORDERED_ACK_ID =
@@ -398,6 +404,62 @@ public void testSendAckOperationsExactlyOnceEnabledMessageFuturesAcks() {
398404
}
399405
}
400406

407+
@Test
408+
public void testSendAckOperationsExactlyOnceEnabledErrorWithEmptyMetadataMap() {
409+
// Setup
410+
411+
// The list(s) of ackIds allows us to mock the grpc response(s)
412+
List<String> ackIdsRequest = new ArrayList<>();
413+
List<AckRequestData> ackRequestDataList = new ArrayList<AckRequestData>();
414+
415+
// Initial) INTERNAL error, retryable
416+
// Retry) PERMISSION_DENIED, not retryable
417+
SettableApiFuture<AckResponse> messageInternalErrorThenPermissionDenied =
418+
SettableApiFuture.create();
419+
ackRequestDataList.add(
420+
AckRequestData.newBuilder(MOCK_ACK_ID_NO_METADATA_MAP_INTERNAL_ERROR_THEN_PERMISSION_DENIED)
421+
.setMessageFuture(messageInternalErrorThenPermissionDenied)
422+
.build());
423+
ackIdsRequest.add(MOCK_ACK_ID_NO_METADATA_MAP_INTERNAL_ERROR_THEN_PERMISSION_DENIED);
424+
425+
// Build our request so we can set our mock responses
426+
AcknowledgeRequest acknowledgeRequest =
427+
AcknowledgeRequest.newBuilder()
428+
.setSubscription(MOCK_SUBSCRIPTION_NAME)
429+
.addAllAckIds(ackIdsRequest)
430+
.build();
431+
432+
ApiException internalError =
433+
new ApiException("internal", null, GrpcStatusCode.of(Code.INTERNAL), true);
434+
ApiException permissionDeniedError =
435+
new ApiException(
436+
"permission_denied", null, GrpcStatusCode.of(Code.PERMISSION_DENIED), false);
437+
// Set mock grpc responses
438+
when(mockSubscriberStub.acknowledgeCallable().futureCall(acknowledgeRequest))
439+
.thenReturn(ApiFutures.immediateFailedFuture(internalError))
440+
.thenReturn(ApiFutures.immediateFailedFuture(permissionDeniedError));
441+
442+
// Instantiate class and run operation(s)
443+
StreamingSubscriberConnection streamingSubscriberConnection =
444+
getStreamingSubscriberConnection(true);
445+
446+
streamingSubscriberConnection.sendAckOperations(ackRequestDataList);
447+
448+
// Backoff
449+
systemExecutor.advanceTime(Duration.ofMillis(200));
450+
451+
// Assert expected behavior;
452+
verify(mockSubscriberStub.acknowledgeCallable(), times(2)).futureCall(acknowledgeRequest);
453+
verify(mockSubscriberStub, never()).modifyAckDeadlineCallable();
454+
455+
try {
456+
assertEquals(AckResponse.PERMISSION_DENIED, messageInternalErrorThenPermissionDenied.get());
457+
} catch (InterruptedException | ExecutionException e) {
458+
// In case something goes wrong retrieving the futures
459+
throw new AssertionError();
460+
}
461+
}
462+
401463
@Test
402464
public void testSetFailureResponseOutstandingMessages() {
403465
// Setup

0 commit comments

Comments
 (0)