Skip to content
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.

Commit 8af2e54

Browse files
committedMay 21, 2023
WIP, will force push later
1 parent 1ae1ded commit 8af2e54

11 files changed

+134
-66
lines changed
 

‎src/main/java/com/rabbitmq/stream/AbstractFlowControlStrategy.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -5,12 +5,12 @@
55
import java.util.Objects;
66
import java.util.function.Supplier;
77

8-
public class AbstractFlowControlStrategy implements ConsumerFlowControlStrategy {
8+
public abstract class AbstractFlowControlStrategy implements ConsumerFlowControlStrategy {
99

10-
private Supplier<Client> clientSupplier;
10+
private final Supplier<Client> clientSupplier;
1111
private volatile Client client;
1212

13-
public AbstractFlowControlStrategy(Supplier<Client> clientSupplier) {
13+
protected AbstractFlowControlStrategy(Supplier<Client> clientSupplier) {
1414
this.clientSupplier = Objects.requireNonNull(clientSupplier, "clientSupplier");
1515
}
1616

‎src/main/java/com/rabbitmq/stream/ClientDataHandler.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,8 @@
33
import com.rabbitmq.stream.impl.Client;
44

55
/**
6-
* Exposes callbacks to handle events from a specific Client, with a leaner interface.
6+
* Exposes callbacks to handle events from a particular {@link Client},
7+
* with specific names for methods and no {@link Client} parameter.
78
*/
89
public interface ClientDataHandler extends
910
Client.PublishConfirmListener,

‎src/main/java/com/rabbitmq/stream/ConsumerFlowControlStrategy.java

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,13 +2,41 @@
22

33
import com.rabbitmq.stream.impl.Client;
44

5+
import java.util.Map;
6+
57
/**
68
* A built and configured flow control strategy for consumers.
79
* Implementations may freely implement reactions to the various client callbacks.
810
* When defined by each implementation, it may internally call {@link Client#credit} to ask for credits.
911
*/
1012
public interface ConsumerFlowControlStrategy extends ClientDataHandler, AutoCloseable {
1113

14+
/**
15+
* Callback for handling a new stream subscription.
16+
* Called right before the subscription is sent to the actual client.
17+
*
18+
* @param subscriptionId The subscriptionId as specified by the Stream Protocol
19+
* @param stream The name of the stream being subscribed to
20+
* @param offsetSpecification The offset specification for this new subscription
21+
* @param subscriptionProperties The subscription properties for this new subscription
22+
* @return The initial credits that should be granted to this new subscription
23+
*/
24+
int handleSubscribe(
25+
byte subscriptionId,
26+
String stream,
27+
OffsetSpecification offsetSpecification,
28+
Map<String, String> subscriptionProperties
29+
);
30+
31+
/**
32+
* Callback for handling a stream unsubscription.
33+
* @param subscriptionId The subscriptionId as specified by the Stream Protocol
34+
*/
35+
default void handleUnsubscribe(byte subscriptionId) {
36+
// No-op by default
37+
}
38+
39+
@Override
1240
default void handleShutdown(Client.ShutdownContext shutdownContext) {
1341
this.close();
1442
}

‎src/main/java/com/rabbitmq/stream/LegacyFlowControlStrategy.java

Lines changed: 22 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22

33
import com.rabbitmq.stream.impl.Client;
44

5+
import java.util.Map;
56
import java.util.function.Supplier;
67

78
/**
@@ -10,13 +11,33 @@
1011
*/
1112
public class LegacyFlowControlStrategy extends AbstractFlowControlStrategy {
1213

14+
private final int initialCredits;
1315
private final int additionalCredits;
1416

15-
public LegacyFlowControlStrategy(Supplier<Client> clientSupplier, int additionalCredits) {
17+
public LegacyFlowControlStrategy(Supplier<Client> clientSupplier) {
18+
this(clientSupplier, 1);
19+
}
20+
21+
public LegacyFlowControlStrategy(Supplier<Client> clientSupplier, int initialCredits) {
22+
this(clientSupplier, initialCredits, 1);
23+
}
24+
25+
public LegacyFlowControlStrategy(Supplier<Client> clientSupplier, int initialCredits, int additionalCredits) {
1626
super(clientSupplier);
27+
this.initialCredits = initialCredits;
1728
this.additionalCredits = additionalCredits;
1829
}
1930

31+
@Override
32+
public int handleSubscribe(
33+
byte subscriptionId,
34+
String stream,
35+
OffsetSpecification offsetSpecification,
36+
Map<String, String> subscriptionProperties
37+
) {
38+
return this.initialCredits;
39+
}
40+
2041
@Override
2142
public void handleChunk(byte subscriptionId, long offset, long messageCount, long dataSize) {
2243
mandatoryClient().credit(subscriptionId, this.additionalCredits);

‎src/main/java/com/rabbitmq/stream/LegacyFlowControlStrategyBuilderFactory.java

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,12 +17,19 @@ public static class LegacyFlowControlStrategyBuilder implements ConsumerFlowCont
1717

1818
private final ConsumerBuilder consumerBuilder;
1919

20+
private int initialCredits = 1;
21+
2022
private int additionalCredits = 1;
2123

2224
public LegacyFlowControlStrategyBuilder(ConsumerBuilder consumerBuilder) {
2325
this.consumerBuilder = consumerBuilder;
2426
}
2527

28+
@Override
29+
public LegacyFlowControlStrategy build(Supplier<Client> clientSupplier) {
30+
return new LegacyFlowControlStrategy(clientSupplier, this.initialCredits, this.additionalCredits);
31+
}
32+
2633
@Override
2734
public ConsumerBuilder builder() {
2835
return this.consumerBuilder;
@@ -33,11 +40,10 @@ public LegacyFlowControlStrategyBuilder additionalCredits(int additionalCredits)
3340
return this;
3441
}
3542

36-
@Override
37-
public LegacyFlowControlStrategy build(Supplier<Client> clientSupplier) {
38-
return new LegacyFlowControlStrategy(clientSupplier, this.additionalCredits);
43+
public LegacyFlowControlStrategyBuilder initialCredits(int initialCredits) {
44+
this.initialCredits = initialCredits;
45+
return this;
3946
}
40-
4147
}
4248

4349
}

‎src/main/java/com/rabbitmq/stream/impl/ConsumersCoordinator.java

Lines changed: 18 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -543,6 +543,7 @@ private class ClientSubscriptionsManager implements Comparable<ClientSubscriptio
543543
// the 2 data structures track the subscriptions, they must remain consistent
544544
private final Map<String, Set<SubscriptionTracker>> streamToStreamSubscriptions =
545545
new ConcurrentHashMap<>();
546+
private final ConsumerFlowControlStrategy consumerFlowControlStrategy;
546547
// trackers and tracker count must be kept in sync
547548
private volatile List<SubscriptionTracker> subscriptionTrackers =
548549
new ArrayList<>(maxConsumersByConnection);
@@ -561,9 +562,10 @@ private ClientSubscriptionsManager(
561562
IntStream.range(0, maxConsumersByConnection).forEach(i -> subscriptionTrackers.add(null));
562563
this.trackerCount = 0;
563564
AtomicReference<Client> clientReference = new AtomicReference<>();
564-
ConsumerFlowControlStrategy consumerFlowControlStrategy = consumerFlowControlStrategyBuilder.build(clientReference::get);
565+
ConsumerFlowControlStrategy localConsumerFlowControlStrategy = consumerFlowControlStrategyBuilder.build(clientReference::get);
566+
this.consumerFlowControlStrategy = localConsumerFlowControlStrategy;
565567
ChunkListener chunkListener = (ignoredClient, subscriptionId, offset, messageCount, dataSize) ->
566-
consumerFlowControlStrategy.handleChunk(subscriptionId, offset, messageCount, dataSize);
568+
localConsumerFlowControlStrategy.handleChunk(subscriptionId, offset, messageCount, dataSize);
567569
CreditNotification creditNotification =
568570
(subscriptionId, responseCode) -> {
569571
SubscriptionTracker subscriptionTracker =
@@ -574,7 +576,7 @@ private ClientSubscriptionsManager(
574576
subscriptionId & 0xFF,
575577
stream,
576578
Utils.formatConstant(responseCode));
577-
consumerFlowControlStrategy.handleCreditNotification(subscriptionId, responseCode);
579+
localConsumerFlowControlStrategy.handleCreditNotification(subscriptionId, responseCode);
578580
};
579581

580582
MessageListener messageListener =
@@ -596,7 +598,7 @@ private ClientSubscriptionsManager(
596598
this.id,
597599
this.name);
598600
}
599-
consumerFlowControlStrategy.handleMessage(subscriptionId, offset, chunkTimestamp, committedOffset, message);
601+
localConsumerFlowControlStrategy.handleMessage(subscriptionId, offset, chunkTimestamp, committedOffset, message);
600602
};
601603
ShutdownListener shutdownListener =
602604
shutdownContext -> {
@@ -650,7 +652,7 @@ private ClientSubscriptionsManager(
650652
"Consumers re-assignment after disconnection from %s",
651653
name));
652654
}
653-
consumerFlowControlStrategy.handleShutdown(shutdownContext);
655+
localConsumerFlowControlStrategy.handleShutdown(shutdownContext);
654656
};
655657
MetadataListener metadataListener =
656658
(stream, code) -> {
@@ -702,7 +704,7 @@ private ClientSubscriptionsManager(
702704
"Consumers re-assignment after metadata update on stream '%s'",
703705
stream));
704706
}
705-
consumerFlowControlStrategy.handleMetadata(stream, code);
707+
localConsumerFlowControlStrategy.handleMetadata(stream, code);
706708
};
707709
ConsumerUpdateListener consumerUpdateListener =
708710
(client, subscriptionId, active) -> {
@@ -721,7 +723,7 @@ private ClientSubscriptionsManager(
721723
LOGGER.debug(
722724
"Could not find stream subscription {} for consumer update", subscriptionId);
723725
}
724-
consumerFlowControlStrategy.handleConsumerUpdate(subscriptionId, active);
726+
localConsumerFlowControlStrategy.handleConsumerUpdate(subscriptionId, active);
725727
return result;
726728
};
727729
String connectionName = connectionNamingStrategy.apply(ClientConnectionType.CONSUMER);
@@ -954,14 +956,20 @@ synchronized void add(
954956

955957
checkNotClosed();
956958
byte subId = subscriptionId;
959+
int initialCredits = this.consumerFlowControlStrategy.handleSubscribe(
960+
subId,
961+
subscriptionTracker.stream,
962+
subscriptionContext.offsetSpecification(),
963+
subscriptionTracker.subscriptionProperties
964+
);
957965
Client.Response subscribeResponse =
958966
Utils.callAndMaybeRetry(
959967
() ->
960968
client.subscribe(
961969
subId,
962970
subscriptionTracker.stream,
963971
subscriptionContext.offsetSpecification(),
964-
subscriptionTracker.initialCredits,
972+
initialCredits,
965973
subscriptionTracker.subscriptionProperties),
966974
RETRY_ON_TIMEOUT,
967975
"Subscribe request for consumer %s on stream '%s'",
@@ -1025,7 +1033,7 @@ synchronized void remove(SubscriptionTracker subscriptionTracker) {
10251033
subscriptionTracker.consumer.id(),
10261034
subscriptionTracker.stream);
10271035
}
1028-
1036+
this.consumerFlowControlStrategy.handleUnsubscribe(subscriptionIdInClient);
10291037
this.setSubscriptionTrackers(update(this.subscriptionTrackers, subscriptionIdInClient, null));
10301038
streamToStreamSubscriptions.compute(
10311039
subscriptionTracker.stream,
@@ -1090,6 +1098,7 @@ synchronized void close() {
10901098
if (this.client != null && this.client.isOpen() && tracker.consumer.isOpen()) {
10911099
this.client.unsubscribe(tracker.subscriptionIdInClient);
10921100
}
1101+
this.consumerFlowControlStrategy.handleUnsubscribe(tracker.subscriptionIdInClient);
10931102
} catch (Exception e) {
10941103
// OK, moving on
10951104
LOGGER.debug(

‎src/main/java/com/rabbitmq/stream/impl/StreamConsumerBuilder.java

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -135,12 +135,23 @@ public ConsumerBuilder noTrackingStrategy() {
135135
return this;
136136
}
137137

138-
public ConsumerBuilder credits(int initial, int onChunkDelivery) {
138+
/**
139+
*
140+
* @param initial Credits to ask for with each new subscription
141+
* @param onChunkDelivery Credits to ask for after a chunk is delivered
142+
* @return this {@link StreamConsumerBuilder}
143+
* @deprecated Prefer using {@link ConsumerBuilder#flowControlStrategy(ConsumerFlowControlStrategyBuilderFactory)}
144+
* to define flow control strategies instead.
145+
*/
146+
@Deprecated
147+
public StreamConsumerBuilder credits(int initial, int onChunkDelivery) {
139148
if (initial <= 0 || onChunkDelivery <= 0) {
140149
throw new IllegalArgumentException("Credits must be positive");
141150
}
142-
this.initialCredits = initial;
143-
this.additionalCredits = onChunkDelivery;
151+
this.consumerFlowControlStrategyBuilder = LegacyFlowControlStrategyBuilderFactory.INSTANCE
152+
.builder(this)
153+
.initialCredits(initial)
154+
.additionalCredits(additionalCredits);
144155
return this;
145156
}
146157

‎src/test/java/com/rabbitmq/stream/Host.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,6 @@ public static String capture(InputStream is) throws IOException {
4343

4444
private static Process executeCommand(String command) throws IOException {
4545
Process pr = executeCommandProcess(command);
46-
4746
int ev = waitForExitValue(pr);
4847
if (ev != 0) {
4948
String stdout = capture(pr.getInputStream());
@@ -83,10 +82,10 @@ private static Process executeCommandProcess(String command) throws IOException
8382
String[] finalCommand;
8483
if (System.getProperty("os.name").toLowerCase().contains("windows")) {
8584
finalCommand = new String[4];
86-
finalCommand[0] = "C:\\winnt\\system32\\cmd.exe";
85+
finalCommand[0] = "C:\\Windows\\system32\\cmd.exe";
8786
finalCommand[1] = "/y";
8887
finalCommand[2] = "/c";
89-
finalCommand[3] = command;
88+
finalCommand[3] = command.replaceAll("\"", "\"\"\"").replaceAll("'", "\"");
9089
} else {
9190
finalCommand = new String[3];
9291
finalCommand[0] = "/bin/sh";

‎src/test/java/com/rabbitmq/stream/impl/ConsumersCoordinatorTest.java

Lines changed: 33 additions & 39 deletions
Original file line numberDiff line numberDiff line change
@@ -33,12 +33,7 @@
3333
import static org.mockito.Mockito.verify;
3434
import static org.mockito.Mockito.when;
3535

36-
import com.rabbitmq.stream.BackOffDelayPolicy;
37-
import com.rabbitmq.stream.Constants;
38-
import com.rabbitmq.stream.OffsetSpecification;
39-
import com.rabbitmq.stream.StreamDoesNotExistException;
40-
import com.rabbitmq.stream.StreamException;
41-
import com.rabbitmq.stream.SubscriptionListener;
36+
import com.rabbitmq.stream.*;
4237
import com.rabbitmq.stream.codec.WrapperMessageBuilder;
4338
import com.rabbitmq.stream.impl.Client.MessageListener;
4439
import com.rabbitmq.stream.impl.Client.QueryOffsetResponse;
@@ -76,8 +71,7 @@ public class ConsumersCoordinatorTest {
7671

7772
private static final SubscriptionListener NO_OP_SUBSCRIPTION_LISTENER = subscriptionContext -> {};
7873
private static final Runnable NO_OP_TRACKING_CLOSING_CALLBACK = () -> {};
79-
private int initialCredits = 10;
80-
private int additionalCredits = 1;
74+
private final int initialCredits = 10;
8175

8276
@Mock StreamEnvironment environment;
8377
@Mock StreamConsumer consumer;
@@ -205,7 +199,7 @@ void tearDown() throws Exception {
205199
NO_OP_SUBSCRIPTION_LISTENER,
206200
NO_OP_TRACKING_CLOSING_CALLBACK,
207201
(offset, message) -> {},
208-
null,
202+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
209203
Collections.emptyMap(),
210204
initialCredits);
211205
verify(clientFactory, times(2)).client(any());
@@ -247,7 +241,7 @@ void shouldGetExactNodeImmediatelyWithAdvertisedHostNameClientFactoryAndExactNod
247241
NO_OP_SUBSCRIPTION_LISTENER,
248242
NO_OP_TRACKING_CLOSING_CALLBACK,
249243
(offset, message) -> {},
250-
null,
244+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
251245
Collections.emptyMap(),
252246
initialCredits);
253247
verify(clientFactory, times(1)).client(any());
@@ -278,7 +272,7 @@ void shouldSubscribeWithEmptyPropertiesWithUnamedConsumer() {
278272
NO_OP_SUBSCRIPTION_LISTENER,
279273
NO_OP_TRACKING_CLOSING_CALLBACK,
280274
(offset, message) -> {},
281-
null,
275+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
282276
Collections.emptyMap(),
283277
initialCredits);
284278
verify(clientFactory, times(1)).client(any());
@@ -300,7 +294,7 @@ void subscribeShouldThrowExceptionWhenNoMetadataForTheStream() {
300294
NO_OP_SUBSCRIPTION_LISTENER,
301295
NO_OP_TRACKING_CLOSING_CALLBACK,
302296
(offset, message) -> {},
303-
null,
297+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
304298
Collections.emptyMap(),
305299
initialCredits))
306300
.isInstanceOf(StreamDoesNotExistException.class);
@@ -320,7 +314,7 @@ void subscribeShouldThrowExceptionWhenStreamDoesNotExist() {
320314
NO_OP_SUBSCRIPTION_LISTENER,
321315
NO_OP_TRACKING_CLOSING_CALLBACK,
322316
(offset, message) -> {},
323-
null,
317+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
324318
Collections.emptyMap(),
325319
initialCredits))
326320
.isInstanceOf(StreamDoesNotExistException.class);
@@ -350,7 +344,7 @@ void subscribePropagateExceptionWhenClientSubscriptionFails() {
350344
NO_OP_SUBSCRIPTION_LISTENER,
351345
NO_OP_TRACKING_CLOSING_CALLBACK,
352346
(offset, message) -> {},
353-
null,
347+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
354348
Collections.emptyMap(),
355349
initialCredits))
356350
.isInstanceOf(StreamException.class)
@@ -372,7 +366,7 @@ void subscribeShouldThrowExceptionWhenMetadataResponseIsNotOk() {
372366
NO_OP_SUBSCRIPTION_LISTENER,
373367
NO_OP_TRACKING_CLOSING_CALLBACK,
374368
(offset, message) -> {},
375-
null,
369+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
376370
Collections.emptyMap(),
377371
initialCredits))
378372
.isInstanceOf(IllegalStateException.class);
@@ -391,7 +385,7 @@ void subscribeShouldThrowExceptionIfNoNodeAvailableForStream() {
391385
NO_OP_SUBSCRIPTION_LISTENER,
392386
NO_OP_TRACKING_CLOSING_CALLBACK,
393387
(offset, message) -> {},
394-
null,
388+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
395389
Collections.emptyMap(),
396390
initialCredits))
397391
.isInstanceOf(IllegalStateException.class);
@@ -433,7 +427,7 @@ void subscribeShouldSubscribeToStreamAndDispatchMessage_UnsubscribeShouldUnsubsc
433427
NO_OP_SUBSCRIPTION_LISTENER,
434428
() -> trackingClosingCallbackCalls.incrementAndGet(),
435429
(offset, message) -> messageHandlerCalls.incrementAndGet(),
436-
null,
430+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
437431
Collections.emptyMap(),
438432
initialCredits);
439433
verify(clientFactory, times(1)).client(any());
@@ -484,7 +478,7 @@ void subscribeShouldSubscribeToStreamAndDispatchMessageWithManySubscriptions() {
484478
NO_OP_TRACKING_CLOSING_CALLBACK,
485479
(offset, message) ->
486480
messageHandlerCalls.compute(subId, (k, v) -> (v == null) ? 1 : ++v),
487-
null,
481+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
488482
Collections.emptyMap(),
489483
initialCredits);
490484
closingRunnables.add(closingRunnable);
@@ -560,7 +554,7 @@ void shouldRedistributeConsumerIfConnectionIsLost() throws Exception {
560554
NO_OP_SUBSCRIPTION_LISTENER,
561555
NO_OP_TRACKING_CLOSING_CALLBACK,
562556
(offset, message) -> messageHandlerCalls.incrementAndGet(),
563-
null,
557+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
564558
Collections.emptyMap(),
565559
initialCredits);
566560
verify(clientFactory, times(1)).client(any());
@@ -580,7 +574,7 @@ void shouldRedistributeConsumerIfConnectionIsLost() throws Exception {
580574
NO_OP_SUBSCRIPTION_LISTENER,
581575
NO_OP_TRACKING_CLOSING_CALLBACK,
582576
(offset, message) -> {},
583-
null,
577+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
584578
Collections.emptyMap(),
585579
initialCredits);
586580

@@ -661,7 +655,7 @@ void shouldSkipRecoveryIfRecoveryIsAlreadyInProgress() throws Exception {
661655
NO_OP_SUBSCRIPTION_LISTENER,
662656
NO_OP_TRACKING_CLOSING_CALLBACK,
663657
(offset, message) -> messageHandlerCalls.incrementAndGet(),
664-
null,
658+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
665659
Collections.emptyMap(),
666660
initialCredits);
667661
verify(clientFactory, times(1)).client(any());
@@ -717,7 +711,7 @@ void shouldRedistributeConsumerOnMetadataUpdate() throws Exception {
717711
NO_OP_SUBSCRIPTION_LISTENER,
718712
NO_OP_TRACKING_CLOSING_CALLBACK,
719713
(offset, message) -> messageHandlerCalls.incrementAndGet(),
720-
null,
714+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
721715
Collections.emptyMap(),
722716
initialCredits);
723717
verify(clientFactory, times(1)).client(any());
@@ -732,7 +726,7 @@ void shouldRedistributeConsumerOnMetadataUpdate() throws Exception {
732726
NO_OP_SUBSCRIPTION_LISTENER,
733727
NO_OP_TRACKING_CLOSING_CALLBACK,
734728
(offset, message) -> {},
735-
null,
729+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
736730
Collections.emptyMap(),
737731
initialCredits);
738732

@@ -821,7 +815,7 @@ void shouldRetryRedistributionIfMetadataIsNotUpdatedImmediately() throws Excepti
821815
NO_OP_SUBSCRIPTION_LISTENER,
822816
NO_OP_TRACKING_CLOSING_CALLBACK,
823817
(offset, message) -> messageHandlerCalls.incrementAndGet(),
824-
null,
818+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
825819
Collections.emptyMap(),
826820
initialCredits);
827821
verify(clientFactory, times(1)).client(any());
@@ -887,7 +881,7 @@ void metadataUpdate_shouldCloseConsumerIfStreamIsDeleted() throws Exception {
887881
NO_OP_SUBSCRIPTION_LISTENER,
888882
NO_OP_TRACKING_CLOSING_CALLBACK,
889883
(offset, message) -> messageHandlerCalls.incrementAndGet(),
890-
null,
884+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
891885
Collections.emptyMap(),
892886
initialCredits);
893887
verify(clientFactory, times(1)).client(any());
@@ -941,7 +935,7 @@ void metadataUpdate_shouldCloseConsumerIfRetryTimeoutIsReached() throws Exceptio
941935
NO_OP_SUBSCRIPTION_LISTENER,
942936
NO_OP_TRACKING_CLOSING_CALLBACK,
943937
(offset, message) -> messageHandlerCalls.incrementAndGet(),
944-
null,
938+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
945939
Collections.emptyMap(),
946940
initialCredits);
947941
verify(clientFactory, times(1)).client(any());
@@ -996,7 +990,7 @@ void shouldUseNewClientsForMoreThanMaxSubscriptionsAndCloseClientAfterUnsubscrip
996990
NO_OP_SUBSCRIPTION_LISTENER,
997991
NO_OP_TRACKING_CLOSING_CALLBACK,
998992
(offset, message) -> {},
999-
null,
993+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
1000994
Collections.emptyMap(),
1001995
initialCredits))
1002996
.collect(Collectors.toList());
@@ -1058,7 +1052,7 @@ void shouldRemoveClientSubscriptionManagerFromPoolAfterConnectionDies() throws E
10581052
NO_OP_SUBSCRIPTION_LISTENER,
10591053
NO_OP_TRACKING_CLOSING_CALLBACK,
10601054
(offset, message) -> {},
1061-
null,
1055+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
10621056
Collections.emptyMap(),
10631057
initialCredits);
10641058
});
@@ -1084,7 +1078,7 @@ void shouldRemoveClientSubscriptionManagerFromPoolAfterConnectionDies() throws E
10841078
NO_OP_SUBSCRIPTION_LISTENER,
10851079
NO_OP_TRACKING_CLOSING_CALLBACK,
10861080
(offset, message) -> {},
1087-
null,
1081+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
10881082
Collections.emptyMap(),
10891083
initialCredits);
10901084

@@ -1125,7 +1119,7 @@ void shouldRemoveClientSubscriptionManagerFromPoolIfEmptyAfterMetadataUpdate() t
11251119
NO_OP_SUBSCRIPTION_LISTENER,
11261120
NO_OP_TRACKING_CLOSING_CALLBACK,
11271121
(offset, message) -> {},
1128-
null,
1122+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
11291123
Collections.emptyMap(),
11301124
initialCredits);
11311125
});
@@ -1157,7 +1151,7 @@ void shouldRemoveClientSubscriptionManagerFromPoolIfEmptyAfterMetadataUpdate() t
11571151
NO_OP_SUBSCRIPTION_LISTENER,
11581152
NO_OP_TRACKING_CLOSING_CALLBACK,
11591153
(offset, message) -> {},
1160-
null,
1154+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
11611155
Collections.emptyMap(),
11621156
initialCredits);
11631157

@@ -1208,7 +1202,7 @@ void shouldRestartWhereItLeftOffAfterDisruption(Consumer<ConsumersCoordinatorTes
12081202
NO_OP_SUBSCRIPTION_LISTENER,
12091203
NO_OP_TRACKING_CLOSING_CALLBACK,
12101204
(offset, message) -> {},
1211-
null,
1205+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
12121206
Collections.emptyMap(),
12131207
initialCredits);
12141208
verify(clientFactory, times(1)).client(any());
@@ -1281,7 +1275,7 @@ void shouldReUseInitialOffsetSpecificationAfterDisruptionIfNoMessagesReceived(
12811275
NO_OP_SUBSCRIPTION_LISTENER,
12821276
NO_OP_TRACKING_CLOSING_CALLBACK,
12831277
(offset, message) -> {},
1284-
null,
1278+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
12851279
Collections.emptyMap(),
12861280
initialCredits);
12871281
verify(clientFactory, times(1)).client(any());
@@ -1356,7 +1350,7 @@ void shouldUseStoredOffsetOnRecovery(Consumer<ConsumersCoordinatorTest> configur
13561350
NO_OP_SUBSCRIPTION_LISTENER,
13571351
NO_OP_TRACKING_CLOSING_CALLBACK,
13581352
(offset, message) -> {},
1359-
null,
1353+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
13601354
Collections.emptyMap(),
13611355
initialCredits);
13621356
verify(clientFactory, times(1)).client(any());
@@ -1443,7 +1437,7 @@ void shouldRetryAssignmentOnRecoveryTimeout() throws Exception {
14431437
NO_OP_SUBSCRIPTION_LISTENER,
14441438
NO_OP_TRACKING_CLOSING_CALLBACK,
14451439
(offset, message) -> {},
1446-
null,
1440+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
14471441
Collections.emptyMap(),
14481442
initialCredits);
14491443
verify(clientFactory, times(1)).client(any());
@@ -1458,7 +1452,7 @@ void shouldRetryAssignmentOnRecoveryTimeout() throws Exception {
14581452
NO_OP_SUBSCRIPTION_LISTENER,
14591453
NO_OP_TRACKING_CLOSING_CALLBACK,
14601454
(offset, message) -> {},
1461-
null,
1455+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
14621456
Collections.emptyMap(),
14631457
initialCredits);
14641458
verify(clientFactory, times(1)).client(any());
@@ -1524,7 +1518,7 @@ void shouldRetryAssignmentOnRecoveryStreamNotAvailableFailure() throws Exception
15241518
NO_OP_SUBSCRIPTION_LISTENER,
15251519
NO_OP_TRACKING_CLOSING_CALLBACK,
15261520
(offset, message) -> {},
1527-
null,
1521+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
15281522
Collections.emptyMap(),
15291523
initialCredits);
15301524
verify(clientFactory, times(1)).client(any());
@@ -1590,7 +1584,7 @@ void shouldRetryAssignmentOnRecoveryCandidateLookupFailure() throws Exception {
15901584
NO_OP_SUBSCRIPTION_LISTENER,
15911585
NO_OP_TRACKING_CLOSING_CALLBACK,
15921586
(offset, message) -> {},
1593-
null,
1587+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
15941588
Collections.emptyMap(),
15951589
initialCredits);
15961590
verify(clientFactory, times(1)).client(any());
@@ -1633,7 +1627,7 @@ void subscribeUnsubscribeInDifferentThreadsShouldNotDeadlock() {
16331627
NO_OP_SUBSCRIPTION_LISTENER,
16341628
NO_OP_TRACKING_CLOSING_CALLBACK,
16351629
(offset, message) -> {},
1636-
null,
1630+
LegacyFlowControlStrategyBuilderFactory.INSTANCE.builder(null),
16371631
Collections.emptyMap(),
16381632
initialCredits);
16391633

‎src/test/java/com/rabbitmq/stream/impl/RetentionClientTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@ static RetentionTestConfig[] retention() {
8080
// small retention in policy
8181
String policyCommand =
8282
String.format(
83-
"set_policy stream-retention-test \"%s\" "
83+
"set_policy stream-retention-test '%s' "
8484
+ "'{\"max-length-bytes\":%d,\"stream-max-segment-size-bytes\":%d }' "
8585
+ "--priority 1 --apply-to queues",
8686
stream, maxLengthBytes, maxSegmentSizeBytes);

‎src/test/java/com/rabbitmq/stream/impl/StreamEnvironmentTest.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -148,8 +148,7 @@ void environmentCreationShouldFailWithUrlUsingWrongPort() {
148148
.build()
149149
.close())
150150
.isInstanceOf(StreamException.class)
151-
.hasCauseInstanceOf(ConnectException.class)
152-
.hasRootCauseMessage("Connection refused");
151+
.hasCauseInstanceOf(ConnectException.class);
153152
}
154153

155154
@Test

0 commit comments

Comments
 (0)
Please sign in to comment.