@Test(expected = IllegalStateException.class) public void invokeExceptionAfterSuccess() { RequestFuture<Void> future = new RequestFuture<>(); future.complete(null); future.exception(); }
/** * Convert from a request future of one type to another type * @param adapter The adapter which does the conversion * @param <S> The type of the future adapted to * @return The new future */ public <S> RequestFuture<S> compose(final RequestFutureAdapter<T, S> adapter) { final RequestFuture<S> adapted = new RequestFuture<>(); addListener(new RequestFutureListener<T>() { @Override public void onSuccess(T value) { adapter.onSuccess(value, adapted); } @Override public void onFailure(RuntimeException e) { adapter.onFailure(e, adapted); } }); return adapted; }
private RequestFuture<ByteBuffer> sendSyncGroupRequest(SyncGroupRequest.Builder requestBuilder) { if (coordinatorUnknown()) return RequestFuture.coordinatorNotAvailable(); return client.send(coordinator, requestBuilder) .compose(new SyncGroupResponseHandler()); }
public static <T> RequestFuture<T> failure(RuntimeException e) { RequestFuture<T> future = new RequestFuture<>(); future.raise(e); return future; }
private ListOffsetResult fetchOffsetsByTimes(Map<TopicPartition, Long> timestampsToSearch, Timer timer, boolean requireTimestamps) { ListOffsetResult result = new ListOffsetResult(); if (timestampsToSearch.isEmpty()) return result; Map<TopicPartition, Long> remainingToSearch = new HashMap<>(timestampsToSearch); do { RequestFuture<ListOffsetResult> future = sendListOffsetsRequests(remainingToSearch, requireTimestamps); client.poll(future, timer); if (!future.isDone()) break; if (future.succeeded()) { ListOffsetResult value = future.value(); result.fetchedOffsets.putAll(value.fetchedOffsets); if (value.partitionsToRetry.isEmpty()) return result; remainingToSearch.keySet().retainAll(value.partitionsToRetry); } else if (!future.isRetriable()) { throw future.exception(); } if (metadata.updateRequested()) client.awaitMetadataUpdate(timer); else timer.sleep(retryBackoffMs); } while (timer.notExpired()); throw new TimeoutException("Failed to get offsets by times in " + timer.elapsedMs() + "ms"); }
@Test public void testComposeSuccessCase() { RequestFuture<String> future = new RequestFuture<>(); RequestFuture<Integer> composed = future.compose(new RequestFutureAdapter<String, Integer>() { @Override public void onSuccess(String value, RequestFuture<Integer> future) { future.complete(value.length()); } }); future.complete("hello"); assertTrue(composed.isDone()); assertTrue(composed.succeeded()); assertEquals(5, (int) composed.value()); }
@Test public void testComposeFailureCase() { RequestFuture<String> future = new RequestFuture<>(); RequestFuture<Integer> composed = future.compose(new RequestFutureAdapter<String, Integer>() { @Override public void onSuccess(String value, RequestFuture<Integer> future) { future.complete(value.length()); } }); RuntimeException e = new RuntimeException(); future.raise(e); assertTrue(composed.isDone()); assertTrue(composed.failed()); assertEquals(e, composed.exception()); }
@Test public void testCoordinatorDisconnect() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // coordinator disconnect will mark coordinator as unknown time.sleep(sessionTimeoutMs); RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat assertEquals(1, consumerClient.pendingRequestCount()); assertFalse(future.isDone()); client.prepareResponse(heartbeatResponse(Errors.NONE), true); // return disconnected time.sleep(sessionTimeoutMs); consumerClient.poll(time.timer(0)); assertTrue(future.isDone()); assertTrue(future.failed()); assertTrue(future.exception() instanceof DisconnectException); assertTrue(coordinator.coordinatorUnknown()); }
if (future.succeeded()) { if (interceptors != null) interceptors.onCommit(offsets); if (future.failed() && !future.isRetriable()) throw future.exception();
@Test public void testBasicCompletion() { RequestFuture<String> future = new RequestFuture<>(); String value = "foo"; future.complete(value); assertTrue(future.isDone()); assertEquals(value, future.value()); }
@Override public void handle(LeaveGroupResponse leaveResponse, RequestFuture<Void> future) { Errors error = leaveResponse.error(); if (error == Errors.NONE) { log.debug("LeaveGroup request returned successfully"); future.complete(null); } else { log.debug("LeaveGroup request failed with error: {}", error.message()); future.raise(error); } } }
@Test public void listenerInvokedIfAddedAfterFutureCompletion() { RequestFuture<Void> future = new RequestFuture<>(); future.complete(null); MockRequestFutureListener<Void> listener = new MockRequestFutureListener<>(); future.addListener(listener); assertOnSuccessInvoked(listener); }
@Test public void listenerInvokedIfAddedBeforeFutureFailure() { RequestFuture<Void> future = new RequestFuture<>(); MockRequestFutureListener<Void> listener = new MockRequestFutureListener<>(); future.addListener(listener); future.raise(new RuntimeException()); assertOnFailureInvoked(listener); }
@Test public void testBasicFailure() { RequestFuture<String> future = new RequestFuture<>(); RuntimeException exception = new RuntimeException(); future.raise(exception); assertTrue(future.isDone()); assertEquals(exception, future.exception()); }
@Test public void testDisconnectWakesUpPoll() throws Exception { final RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat()); client.enableBlockingUntilWakeup(1); Thread t = new Thread() { @Override public void run() { consumerClient.poll(future); } }; t.start(); consumerClient.disconnectAsync(node); t.join(); assertTrue(future.failed()); assertTrue(future.exception() instanceof DisconnectException); }
@Test(expected = IllegalStateException.class) public void invokeCompleteAfterAlreadyFailed() { RequestFuture<Void> future = new RequestFuture<>(); future.raise(new RuntimeException()); future.complete(null); }
@Test public void send() { client.prepareResponse(heartbeatResponse(Errors.NONE)); RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat()); assertEquals(1, consumerClient.pendingRequestCount()); assertEquals(1, consumerClient.pendingRequestCount(node)); assertFalse(future.isDone()); consumerClient.poll(future); assertTrue(future.isDone()); assertTrue(future.succeeded()); ClientResponse clientResponse = future.value(); HeartbeatResponse response = (HeartbeatResponse) clientResponse.responseBody(); assertEquals(Errors.NONE, response.error()); }
/** * Add a listener which will be notified when the future completes * @param listener non-null listener to add */ public void addListener(RequestFutureListener<T> listener) { this.listeners.add(listener); if (failed()) fireFailure(); else if (succeeded()) fireSuccess(); }