2121import java .util .concurrent .TimeUnit ;
2222import java .util .concurrent .TimeoutException ;
2323import java .util .concurrent .atomic .AtomicInteger ;
24+ import java .util .concurrent .atomic .AtomicReference ;
2425
2526import javax .annotation .Nullable ;
2627
2930import com .google .common .util .concurrent .ListenableFuture ;
3031import com .google .common .util .concurrent .MoreExecutors ;
3132
33+ import org .awaitility .Awaitility ;
3234import org .junit .After ;
3335import org .junit .Before ;
3436import org .junit .BeforeClass ;
6769import org .apache .cassandra .utils .MockFailureDetector ;
6870
6971import static java .util .concurrent .TimeUnit .MINUTES ;
72+ import static java .util .concurrent .TimeUnit .SECONDS ;
7073import static org .apache .cassandra .Util .spinAssertEquals ;
7174import static org .apache .cassandra .config .CassandraRelevantProperties .HINT_DISPATCH_INTERVAL_MS ;
7275import static org .apache .cassandra .hints .HintsTestUtil .sendHintsAndResponses ;
@@ -139,14 +142,15 @@ public void testDispatchHints() throws InterruptedException, ExecutionException
139142 long cnt = StorageMetrics .totalHints .getCount ();
140143
141144 // create spy for hint messages
142- MockMessagingSpy spy = sendHintsAndResponses (metadata , 100 , -1 );
143-
144- // metrics should have been updated with number of create hints
145- assertEquals (cnt + 100 , StorageMetrics .totalHints .getCount ());
146-
147- // wait until hints have been send
148- spy .interceptMessageOut (100 ).get ();
149- spy .interceptNoMsg (500 , TimeUnit .MILLISECONDS ).get ();
145+ try (MockMessagingSpy spy = sendHintsAndResponses (metadata , 100 , -1 ))
146+ {
147+ // metrics should have been updated with number of create hints
148+ assertEquals (cnt + 100 , StorageMetrics .totalHints .getCount ());
149+
150+ // wait until hints have been send
151+ spy .interceptMessageOut (100 ).get ();
152+ spy .interceptNoMsg (500 , TimeUnit .MILLISECONDS ).get ();
153+ }
150154 }
151155
152156 @ Test
@@ -155,66 +159,78 @@ public void testPauseAndResume() throws InterruptedException, ExecutionException
155159 HintsService .instance .pauseDispatch ();
156160
157161 // create spy for hint messages
158- MockMessagingSpy spy = sendHintsAndResponses (metadata , 100 , -1 );
159-
160- // we should not send any hints while paused
161- ListenableFuture <Boolean > noMessagesWhilePaused = spy .interceptNoMsg (15 , TimeUnit .SECONDS );
162- Futures .addCallback (noMessagesWhilePaused , new MoreFutures .SuccessCallback <Boolean >()
162+ try (MockMessagingSpy spy = sendHintsAndResponses (metadata , 100 , -1 ))
163163 {
164- public void onSuccess (@ Nullable Boolean aBoolean )
164+
165+ // we should not send any hints while paused
166+ ListenableFuture <Boolean > noMessagesWhilePaused = spy .interceptNoMsg (15 , TimeUnit .SECONDS );
167+ Futures .addCallback (noMessagesWhilePaused , new MoreFutures .SuccessCallback <Boolean >()
165168 {
166- HintsService .instance .resumeDispatch ();
167- }
168- }, MoreExecutors .directExecutor ());
169-
170- Futures .allAsList (
171- noMessagesWhilePaused ,
172- spy .interceptMessageOut (100 ),
173- spy .interceptNoMsg (200 , TimeUnit .MILLISECONDS )
174- ).get ();
169+ public void onSuccess (@ Nullable Boolean aBoolean )
170+ {
171+ HintsService .instance .resumeDispatch ();
172+ }
173+ }, MoreExecutors .directExecutor ());
174+
175+ Futures .allAsList (
176+ noMessagesWhilePaused ,
177+ spy .interceptMessageOut (100 ),
178+ spy .interceptNoMsg (200 , TimeUnit .MILLISECONDS )
179+ ).get ();
180+ }
175181 }
176182
177183 @ Test
178184 public void testPageRetry () throws InterruptedException , ExecutionException , TimeoutException
179185 {
180186 // create spy for hint messages, but only create responses for 5 hints
181- MockMessagingSpy spy = sendHintsAndResponses (metadata , 20 , 5 );
187+ try (MockMessagingSpy spy = sendHintsAndResponses (metadata , 20 , 5 ))
188+ {
182189
183- Futures .allAsList (
184- // the dispatcher will always send all hints within the current page
185- // and only wait for the acks before going to the next page
186- spy .interceptMessageOut (20 ),
187- spy .interceptNoMsg (200 , TimeUnit .MILLISECONDS ),
190+ Futures .allAsList (
191+ // the dispatcher will always send all hints within the current page
192+ // and only wait for the acks before going to the next page
193+ spy .interceptMessageOut (20 ),
194+ spy .interceptNoMsg (200 , TimeUnit .MILLISECONDS ),
188195
189- // next tick will trigger a retry of the same page as we only replied with 5/20 acks
190- spy .interceptMessageOut (20 )
191- ).get ();
196+ // next tick will trigger a retry of the same page as we only replied with 5/20 acks
197+ spy .interceptMessageOut (20 )
198+ ).get ();
192199
193- // marking the destination node as dead should stop sending hints
194- failureDetector .isAlive = false ;
195- spy .interceptNoMsg (20 , TimeUnit .SECONDS ).get ();
200+ // marking the destination node as dead should stop sending hints
201+ failureDetector .isAlive = false ;
202+ spy .interceptNoMsg (20 , TimeUnit .SECONDS ).get ();
203+ }
196204 }
197205
198206 @ Test
199207 public void testPageSeek () throws InterruptedException , ExecutionException
200208 {
201209 // create spy for hint messages, stop replying after 12k (should be on 3rd page)
202- MockMessagingSpy spy = sendHintsAndResponses (metadata , 20000 , 12000 );
203-
204- // At this point the dispatcher will constantly retry the page we stopped acking,
205- // thus we receive the same hints from the page multiple times and in total more than
206- // all written hints. Lets just consume them for a while and then pause the dispatcher.
207- spy .interceptMessageOut (22000 ).get ();
208- HintsService .instance .pauseDispatch ();
209- Thread .sleep (1000 );
210-
211- // verify that we have a dispatch offset set for the page we're currently stuck at
212- HintsStore store = HintsService .instance .getCatalog ().get (StorageService .instance .getLocalHostUUID ());
213- HintsDescriptor descriptor = store .poll ();
214- store .offerFirst (descriptor ); // add again for cleanup during re-instanciation
215- InputPosition dispatchOffset = store .getDispatchOffset (descriptor );
216- assertTrue (dispatchOffset != null );
217- assertTrue (((ChecksummedDataInput .Position ) dispatchOffset ).sourcePosition > 0 );
210+ try (MockMessagingSpy spy = sendHintsAndResponses (metadata , 20000 , 12000 ))
211+ {
212+ // At this point the dispatcher will constantly retry the page we stopped acking,
213+ // thus we receive the same hints from the page multiple times and in total more than
214+ // all written hints. Lets just consume them for a while and then pause the dispatcher.
215+ spy .interceptMessageOut (22000 ).get ();
216+ HintsService .instance .pauseDispatch ();
217+ Thread .sleep (1000 );
218+
219+ // verify that we have a dispatch offset set for the page we're currently stuck at
220+ HintsStore store = HintsService .instance .getCatalog ().get (StorageService .instance .getLocalHostUUID ());
221+ AtomicReference <HintsDescriptor > hintDescriptorRef = new AtomicReference <>();
222+ Awaitility .waitAtMost (20 , SECONDS ).until (() -> {
223+ HintsDescriptor descriptor = store .poll ();
224+ if (descriptor != null )
225+ hintDescriptorRef .set (descriptor );
226+ return descriptor != null ;
227+ });
228+ HintsDescriptor descriptor = hintDescriptorRef .get ();
229+ store .offerFirst (descriptor ); // add again for cleanup during re-instanciation
230+ InputPosition dispatchOffset = store .getDispatchOffset (descriptor );
231+ assertTrue (dispatchOffset != null );
232+ assertTrue (((ChecksummedDataInput .Position ) dispatchOffset ).sourcePosition > 0 );
233+ }
218234 }
219235
220236 /*
0 commit comments