23
23
import java .io .PrintWriter ;
24
24
import java .util .Set ;
25
25
import java .util .concurrent .ConcurrentHashMap ;
26
- import java .util .concurrent .CountDownLatch ;
27
26
import java .util .concurrent .ExecutionException ;
28
27
import java .util .concurrent .ExecutorService ;
29
28
import java .util .concurrent .Executors ;
36
35
import org .apache .beam .vendor .grpc .v1p69p0 .io .grpc .stub .CallStreamObserver ;
37
36
import org .apache .beam .vendor .grpc .v1p69p0 .io .grpc .stub .StreamObserver ;
38
37
import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .util .concurrent .Uninterruptibles ;
38
+ import org .joda .time .Duration ;
39
+ import org .joda .time .Instant ;
39
40
import org .junit .Before ;
40
41
import org .junit .Test ;
41
42
import org .junit .runner .RunWith ;
42
43
import org .junit .runners .JUnit4 ;
44
+ import org .slf4j .Logger ;
43
45
import org .slf4j .LoggerFactory ;
44
46
45
47
@ RunWith (JUnit4 .class )
@@ -61,61 +63,83 @@ private TestStream newStream(
61
63
62
64
@ Test
63
65
public void testShutdown_notBlockedBySend () throws InterruptedException , ExecutionException {
64
- CountDownLatch sendBlocker = new CountDownLatch ( 1 );
66
+ TestCallStreamObserver callStreamObserver = TestCallStreamObserver . notReady ( );
65
67
Function <StreamObserver <Integer >, StreamObserver <Integer >> clientFactory =
66
- ignored ->
67
- new CallStreamObserver <Integer >() {
68
- @ Override
69
- public void onNext (Integer integer ) {
70
- try {
71
- sendBlocker .await ();
72
- } catch (InterruptedException e ) {
73
- throw new RuntimeException (e );
74
- }
75
- }
68
+ ignored -> callStreamObserver ;
76
69
77
- @ Override
78
- public void onError (Throwable throwable ) {}
70
+ TestStream testStream = newStream (clientFactory );
71
+ testStream .start ();
72
+ ExecutorService sendExecutor = Executors .newSingleThreadExecutor ();
73
+ Future <WindmillStreamShutdownException > sendFuture =
74
+ sendExecutor .submit (
75
+ () -> {
76
+ // Send a few times to trigger blocking in the CallStreamObserver.
77
+ testStream .testSend ();
78
+ testStream .testSend ();
79
+ return assertThrows (WindmillStreamShutdownException .class , testStream ::testSend );
80
+ });
79
81
80
- @ Override
81
- public void onCompleted () {}
82
+ // Wait for 1 send since it always goes through, the rest may buffer.
83
+ callStreamObserver . waitForSends ( 1 );
82
84
83
- @ Override
84
- public boolean isReady () {
85
- return false ;
86
- }
85
+ testStream .shutdown ();
87
86
88
- @ Override
89
- public void setOnReadyHandler ( Runnable runnable ) { }
87
+ assertThat ( sendFuture . get ()). isInstanceOf ( WindmillStreamShutdownException . class );
88
+ }
90
89
91
- @ Override
92
- public void disableAutoInboundFlowControl () {}
90
+ @ Test
91
+ public void testMaybeScheduleHealthCheck () {
92
+ TestCallStreamObserver callStreamObserver = TestCallStreamObserver .create ();
93
+ Function <StreamObserver <Integer >, StreamObserver <Integer >> clientFactory =
94
+ ignored -> callStreamObserver ;
93
95
94
- @ Override
95
- public void request (int i ) {}
96
+ TestStream testStream = newStream (clientFactory );
97
+ testStream .start ();
98
+ Instant reportingThreshold = Instant .now ().minus (Duration .millis (1 ));
96
99
97
- @ Override
98
- public void setMessageCompression (boolean b ) {}
99
- };
100
+ testStream .maybeScheduleHealthCheck (reportingThreshold );
101
+ testStream .waitForHealthChecks (1 );
102
+ assertThat (testStream .numHealthChecks .get ()).isEqualTo (1 );
103
+ testStream .shutdown ();
104
+ }
105
+
106
+ @ Test
107
+ public void testMaybeSendHealthCheck_doesNotSendIfLastScheduleLessThanThreshold () {
108
+ TestCallStreamObserver callStreamObserver = TestCallStreamObserver .create ();
109
+ Function <StreamObserver <Integer >, StreamObserver <Integer >> clientFactory =
110
+ ignored -> callStreamObserver ;
100
111
101
112
TestStream testStream = newStream (clientFactory );
102
113
testStream .start ();
103
- ExecutorService sendExecutor = Executors .newSingleThreadExecutor ();
104
- Future <WindmillStreamShutdownException > sendFuture =
105
- sendExecutor .submit (
106
- () ->
107
- assertThrows (WindmillStreamShutdownException .class , () -> testStream .testSend (1 )));
108
- testStream .shutdown ();
109
114
110
- // Sleep a bit to give sendExecutor time to execute the send().
115
+ try {
116
+ testStream .trySend (1 );
117
+ } catch (WindmillStreamShutdownException e ) {
118
+ throw new RuntimeException (e );
119
+ }
120
+
121
+ // Set a really long reporting threshold.
122
+ Instant reportingThreshold = Instant .now ().minus (Duration .standardHours (1 ));
123
+
124
+ // Should not send health checks since we just sent the above message.
125
+ testStream .maybeScheduleHealthCheck (reportingThreshold );
126
+ Uninterruptibles .sleepUninterruptibly (100 , TimeUnit .MILLISECONDS );
127
+ testStream .maybeScheduleHealthCheck (reportingThreshold );
111
128
Uninterruptibles .sleepUninterruptibly (100 , TimeUnit .MILLISECONDS );
112
129
113
- sendBlocker .countDown ();
114
- assertThat (sendFuture .get ()).isInstanceOf (WindmillStreamShutdownException .class );
130
+ callStreamObserver .waitForSends (1 );
131
+ // Sleep just to ensure an async health check doesn't show up
132
+ Uninterruptibles .sleepUninterruptibly (100 , TimeUnit .MILLISECONDS );
133
+
134
+ assertThat (testStream .numHealthChecks .get ()).isEqualTo (0 );
135
+ testStream .shutdown ();
115
136
}
116
137
117
138
private static class TestStream extends AbstractWindmillStream <Integer , Integer > {
139
+ private static final Logger LOG = LoggerFactory .getLogger (AbstractWindmillStreamTest .class );
140
+
118
141
private final AtomicInteger numStarts = new AtomicInteger ();
142
+ private final AtomicInteger numHealthChecks = new AtomicInteger ();
119
143
120
144
private TestStream (
121
145
Function <StreamObserver <Integer >, StreamObserver <Integer >> clientFactory ,
@@ -148,19 +172,87 @@ protected boolean hasPendingRequests() {
148
172
@ Override
149
173
protected void startThrottleTimer () {}
150
174
151
- public void testSend (Integer i )
152
- throws ResettableThrowingStreamObserver .StreamClosedException ,
153
- WindmillStreamShutdownException {
154
- trySend (i );
175
+ private void testSend () throws WindmillStreamShutdownException {
176
+ trySend (1 );
155
177
}
156
178
157
179
@ Override
158
- protected void sendHealthCheck () {}
180
+ protected void sendHealthCheck () {
181
+ numHealthChecks .incrementAndGet ();
182
+ }
183
+
184
+ private void waitForHealthChecks (int expectedHealthChecks ) {
185
+ int waitedMillis = 0 ;
186
+ while (numHealthChecks .get () < expectedHealthChecks ) {
187
+ LOG .info (
188
+ "Waited for {}ms for {} health checks. Current health check count is {}." ,
189
+ waitedMillis ,
190
+ numHealthChecks .get (),
191
+ expectedHealthChecks );
192
+ Uninterruptibles .sleepUninterruptibly (100 , TimeUnit .MILLISECONDS );
193
+ }
194
+ }
159
195
160
196
@ Override
161
197
protected void appendSpecificHtml (PrintWriter writer ) {}
162
198
163
199
@ Override
164
200
protected void shutdownInternal () {}
165
201
}
202
+
203
+ private static class TestCallStreamObserver extends CallStreamObserver <Integer > {
204
+ private static final Logger LOG = LoggerFactory .getLogger (AbstractWindmillStreamTest .class );
205
+ private final AtomicInteger numSends = new AtomicInteger ();
206
+ private final boolean isReady ;
207
+
208
+ private TestCallStreamObserver (boolean isReady ) {
209
+ this .isReady = isReady ;
210
+ }
211
+
212
+ private static TestCallStreamObserver create () {
213
+ return new TestCallStreamObserver (true );
214
+ }
215
+
216
+ private static TestCallStreamObserver notReady () {
217
+ return new TestCallStreamObserver (false );
218
+ }
219
+
220
+ @ Override
221
+ public void onNext (Integer integer ) {
222
+ numSends .incrementAndGet ();
223
+ }
224
+
225
+ private void waitForSends (int expectedSends ) {
226
+ int millisWaited = 0 ;
227
+ while (numSends .get () < expectedSends ) {
228
+ LOG .info (
229
+ "Waited {}ms for {} sends, current sends: {}" , millisWaited , expectedSends , numSends );
230
+ Uninterruptibles .sleepUninterruptibly (100 , TimeUnit .MILLISECONDS );
231
+ millisWaited += 100 ;
232
+ }
233
+ }
234
+
235
+ @ Override
236
+ public void onError (Throwable throwable ) {}
237
+
238
+ @ Override
239
+ public void onCompleted () {}
240
+
241
+ @ Override
242
+ public boolean isReady () {
243
+ return isReady ;
244
+ }
245
+
246
+ @ Override
247
+ public void setOnReadyHandler (Runnable runnable ) {}
248
+
249
+ @ Override
250
+ public void disableAutoInboundFlowControl () {}
251
+
252
+ @ Override
253
+ public void request (int i ) {}
254
+
255
+ @ Override
256
+ public void setMessageCompression (boolean b ) {}
257
+ }
166
258
}
0 commit comments