Skip to content

Commit f4ca4b2

Browse files
fix: do not block forever if message size > flow control limits (#847)
* Modifying Publish example in README to match other examples given, and fix issue #6784 * fix: Modifying Publish example in README to match other examples, and fix Issue #11 * feat: Adding support for DLQs Adding delivery attempt count to PubsubMessages as a message attribute, and creating helper function to allow users to get the count without knowing implementation details. * Fix formatting * fix: making changes requested in pull request * fix: creating fix to not populate delivery attempt attribute when dead lettering is not enabled * Adding unit test for case in which a received message has no delivery attempt * Making MessageWaiter class more generic to also be used for outstanding ack operations * Waiting for acks to complete before shutting down a streaming subscriber connection * Fixing formatting error * fix: making sure all publishes complete before shutting down the publisher * adding default max outstanding request bytes * fix: make push endpoint valid https * fix: use default zero value if a flow control setting is not provided * fix lint issues * fix: better cleanup during publisher test * fix: format issues * fix: test timeouts should be a minute * fix: make subscriberIt less flaky * fix: add deprecated tag for deprecated methods * fix: retrying sync pulls in subscriberIT test * Revert "fix: retrying sync pulls in subscriberIT test" This reverts commit fef9956. * fix: do not send ModAck if auto ack extensions are disabled * Adding test for zero ack extension * revert maxAckExtension changes * fix: flow control blocks forever * fix: throw flow control exception when message byte size is too big
1 parent 374b8c3 commit f4ca4b2

File tree

2 files changed

+109
-24
lines changed

2 files changed

+109
-24
lines changed

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

+12
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import com.google.api.gax.batching.BatchingSettings;
2828
import com.google.api.gax.batching.FlowControlSettings;
2929
import com.google.api.gax.batching.FlowController;
30+
import com.google.api.gax.batching.FlowController.LimitExceededBehavior;
3031
import com.google.api.gax.core.BackgroundResource;
3132
import com.google.api.gax.core.BackgroundResourceAggregation;
3233
import com.google.api.gax.core.CredentialsProvider;
@@ -779,6 +780,11 @@ public Builder setBatchingSettings(BatchingSettings batchingSettings) {
779780
Preconditions.checkArgument(batchingSettings.getRequestByteThreshold() > 0);
780781
Preconditions.checkNotNull(batchingSettings.getDelayThreshold());
781782
Preconditions.checkArgument(batchingSettings.getDelayThreshold().toMillis() > 0);
783+
FlowControlSettings flowControlSettings = batchingSettings.getFlowControlSettings();
784+
if (flowControlSettings.getLimitExceededBehavior() != LimitExceededBehavior.Ignore) {
785+
Preconditions.checkArgument(flowControlSettings.getMaxOutstandingElementCount() > 0);
786+
Preconditions.checkArgument(flowControlSettings.getMaxOutstandingRequestBytes() > 0);
787+
}
782788
this.batchingSettings = batchingSettings;
783789
return this;
784790
}
@@ -859,6 +865,12 @@ private static class MessageFlowController {
859865
}
860866

861867
void acquire(long messageSize) throws FlowController.FlowControlException {
868+
if (messageSize > byteLimit) {
869+
logger.log(
870+
Level.WARNING,
871+
"Attempted to publish message with byte size > request byte flow control limit.");
872+
throw new FlowController.MaxOutstandingRequestBytesReachedException(byteLimit);
873+
}
862874
lock.lock();
863875
try {
864876
if (outstandingMessages >= messageLimit

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

+97-24
Original file line numberDiff line numberDiff line change
@@ -1019,7 +1019,57 @@ public void testShutDown() throws Exception {
10191019
}
10201020

10211021
@Test
1022-
public void testPublishFlowControl_throwException() throws Exception {
1022+
public void invalidFlowControlBytes_throwException() throws Exception {
1023+
try {
1024+
Publisher publisher =
1025+
getTestPublisherBuilder()
1026+
.setBatchingSettings(
1027+
Publisher.Builder.DEFAULT_BATCHING_SETTINGS
1028+
.toBuilder()
1029+
.setElementCountThreshold(1L)
1030+
.setDelayThreshold(Duration.ofSeconds(5))
1031+
.setFlowControlSettings(
1032+
FlowControlSettings.newBuilder()
1033+
.setLimitExceededBehavior(
1034+
FlowController.LimitExceededBehavior.ThrowException)
1035+
.setMaxOutstandingElementCount(1L)
1036+
.setMaxOutstandingRequestBytes(0L)
1037+
.build())
1038+
.build())
1039+
.build();
1040+
fail("Expected an IllegalArgumentException");
1041+
} catch (Exception e) {
1042+
assertThat(e).isInstanceOf(IllegalArgumentException.class);
1043+
}
1044+
}
1045+
1046+
@Test
1047+
public void invalidFlowControlElementCount_throwException() throws Exception {
1048+
try {
1049+
Publisher publisher =
1050+
getTestPublisherBuilder()
1051+
.setBatchingSettings(
1052+
Publisher.Builder.DEFAULT_BATCHING_SETTINGS
1053+
.toBuilder()
1054+
.setElementCountThreshold(1L)
1055+
.setDelayThreshold(Duration.ofSeconds(5))
1056+
.setFlowControlSettings(
1057+
FlowControlSettings.newBuilder()
1058+
.setLimitExceededBehavior(
1059+
FlowController.LimitExceededBehavior.ThrowException)
1060+
.setMaxOutstandingElementCount(0L)
1061+
.setMaxOutstandingRequestBytes(1000L)
1062+
.build())
1063+
.build())
1064+
.build();
1065+
fail("Expected an IllegalArgumentException");
1066+
} catch (Exception e) {
1067+
assertThat(e).isInstanceOf(IllegalArgumentException.class);
1068+
}
1069+
}
1070+
1071+
@Test
1072+
public void testMessageExceedsFlowControlLimits_throwException() throws Exception {
10231073
Publisher publisher =
10241074
getTestPublisherBuilder()
10251075
.setExecutorProvider(SINGLE_THREAD_EXECUTOR)
@@ -1030,39 +1080,57 @@ public void testPublishFlowControl_throwException() throws Exception {
10301080
.setDelayThreshold(Duration.ofSeconds(5))
10311081
.setFlowControlSettings(
10321082
FlowControlSettings.newBuilder()
1033-
.setLimitExceededBehavior(
1034-
FlowController.LimitExceededBehavior.ThrowException)
1083+
.setLimitExceededBehavior(FlowController.LimitExceededBehavior.Block)
10351084
.setMaxOutstandingElementCount(1L)
1036-
.setMaxOutstandingRequestBytes(10L)
1085+
.setMaxOutstandingRequestBytes(1L)
10371086
.build())
10381087
.build())
10391088
.build();
10401089

1041-
// Sending a message that is too large results in an exception.
1042-
ApiFuture<String> publishFuture1 = sendTestMessage(publisher, "AAAAAAAAAAA");
10431090
try {
1044-
publishFuture1.get();
1045-
fail("Should have thrown an FlowController.MaxOutstandingRequestBytesReachedException");
1091+
sendTestMessage(publisher, "AAAAAAAAAAAAAAAAAAAAA").get();
1092+
fail("Should have thrown a FlowController.MaxOutstandingRequestBytesReachedException");
10461093
} catch (ExecutionException e) {
10471094
assertThat(e.getCause())
10481095
.isInstanceOf(FlowController.MaxOutstandingRequestBytesReachedException.class);
10491096
}
1097+
}
10501098

1051-
// Sending a second message succeeds.
1052-
ApiFuture<String> publishFuture2 = sendTestMessage(publisher, "AAAA");
1099+
@Test
1100+
public void testPublishFlowControl_throwException() throws Exception {
1101+
Publisher publisher =
1102+
getTestPublisherBuilder()
1103+
.setExecutorProvider(SINGLE_THREAD_EXECUTOR)
1104+
.setBatchingSettings(
1105+
Publisher.Builder.DEFAULT_BATCHING_SETTINGS
1106+
.toBuilder()
1107+
.setElementCountThreshold(1L)
1108+
.setDelayThreshold(Duration.ofSeconds(5))
1109+
.setFlowControlSettings(
1110+
FlowControlSettings.newBuilder()
1111+
.setLimitExceededBehavior(
1112+
FlowController.LimitExceededBehavior.ThrowException)
1113+
.setMaxOutstandingElementCount(1L)
1114+
.setMaxOutstandingRequestBytes(10L)
1115+
.build())
1116+
.build())
1117+
.build();
1118+
1119+
// Sending a message less than the byte limit succeeds.
1120+
ApiFuture<String> publishFuture1 = sendTestMessage(publisher, "AAAA");
10531121

1054-
// Sending a third message fails because of the outstanding message.
1055-
ApiFuture<String> publishFuture3 = sendTestMessage(publisher, "AA");
1122+
// Sending another message fails because of the outstanding message.
1123+
ApiFuture<String> publishFuture2 = sendTestMessage(publisher, "AA");
10561124
try {
1057-
publishFuture3.get();
1125+
publishFuture2.get();
10581126
fail("Should have thrown an FlowController.MaxOutstandingElementCountReachedException");
10591127
} catch (ExecutionException e) {
10601128
assertThat(e.getCause())
10611129
.isInstanceOf(FlowController.MaxOutstandingElementCountReachedException.class);
10621130
}
10631131

10641132
testPublisherServiceImpl.addPublishResponse(PublishResponse.newBuilder().addMessageIds("1"));
1065-
assertEquals("1", publishFuture2.get());
1133+
assertEquals("1", publishFuture1.get());
10661134

10671135
// Sending another message succeeds.
10681136
ApiFuture<String> publishFuture4 = sendTestMessage(publisher, "AAAA");
@@ -1091,22 +1159,27 @@ public void testPublishFlowControl_throwExceptionWithOrderingKey() throws Except
10911159
.setEnableMessageOrdering(true)
10921160
.build();
10931161

1094-
// Sending a message that is too large results in an exception.
1095-
ApiFuture<String> publishFuture1 =
1096-
sendTestMessageWithOrderingKey(publisher, "AAAAAAAAAAA", "a");
1162+
// Sending a message less than the byte limit succeeds.
1163+
ApiFuture<String> publishFuture1 = sendTestMessageWithOrderingKey(publisher, "AAAA", "a");
1164+
1165+
// Sending another message fails because of the outstanding message.
1166+
ApiFuture<String> publishFuture2 = sendTestMessageWithOrderingKey(publisher, "AA", "a");
10971167
try {
1098-
publishFuture1.get();
1099-
fail("Should have thrown an FlowController.MaxOutstandingRequestBytesReachedException");
1168+
publishFuture2.get();
1169+
fail("Should have thrown an FlowController.MaxOutstandingElementCountReachedException");
11001170
} catch (ExecutionException e) {
11011171
assertThat(e.getCause())
1102-
.isInstanceOf(FlowController.MaxOutstandingRequestBytesReachedException.class);
1172+
.isInstanceOf(FlowController.MaxOutstandingElementCountReachedException.class);
11031173
}
11041174

1105-
// Sending a second message for the same ordering key fails because the first one failed.
1106-
ApiFuture<String> publishFuture2 = sendTestMessageWithOrderingKey(publisher, "AAAA", "a");
1175+
testPublisherServiceImpl.addPublishResponse(PublishResponse.newBuilder().addMessageIds("1"));
1176+
assertEquals("1", publishFuture1.get());
1177+
1178+
// Sending another message for the same ordering key fails because the one before failed.
1179+
ApiFuture<String> publishFuture3 = sendTestMessageWithOrderingKey(publisher, "AAAA", "a");
11071180
try {
1108-
publishFuture2.get();
1109-
Assert.fail("This should fail.");
1181+
publishFuture3.get();
1182+
fail("This should fail.");
11101183
} catch (ExecutionException e) {
11111184
assertEquals(SequentialExecutorService.CallbackExecutor.CANCELLATION_EXCEPTION, e.getCause());
11121185
}

0 commit comments

Comments
 (0)