/** * Poll for any network IO. * @param timer Timer bounding how long this method can block * @param pollCondition Nullable blocking condition */ public void poll(Timer timer, PollCondition pollCondition) { poll(timer, pollCondition, false); }
/** * Wakeup the consumer. This method is thread-safe and is useful in particular to abort a long poll. * The thread which is blocking in an operation will throw {@link org.apache.kafka.common.errors.WakeupException}. * If no thread is blocking in a method which can throw {@link org.apache.kafka.common.errors.WakeupException}, the next call to such a method will raise it instead. */ @Override public void wakeup() { this.client.wakeup(); }
/** * Ensure our metadata is fresh (if an update is expected, this will block * until it has completed). */ boolean ensureFreshMetadata(Timer timer) { if (this.metadata.updateRequested() || this.metadata.timeToNextUpdate(timer.currentTimeMs()) == 0) { return awaitMetadataUpdate(timer); } else { // the metadata is already fresh return true; } }
/** * Block until all pending requests from the given node have finished. * @param node The node to await requests from * @param timer Timer bounding how long this method can block * @return true If all requests finished, false if the timeout expired first */ public boolean awaitPendingRequests(Node node, Timer timer) { while (hasPendingRequests(node) && timer.notExpired()) { poll(timer); } return !hasPendingRequests(node); }
/** * Send Metadata Request to least loaded node in Kafka cluster asynchronously * @return A future that indicates result of sent metadata request */ private RequestFuture<ClientResponse> sendMetadataRequest(MetadataRequest.Builder request) { final Node node = client.leastLoadedNode(); if (node == null) return RequestFuture.noBrokersAvailable(); else return client.send(node, request); }
protected void close(Timer timer) { try { closeHeartbeatThread(); } finally { // Synchronize after closing the heartbeat thread since heartbeat thread // needs this lock to complete and terminate after close flag is set. synchronized (this) { if (leaveGroupOnClose) { maybeLeaveGroup(); } // At this point, there may be pending commits (async commits or sync commits that were // interrupted using wakeup) and the leave group request which have been queued, but not // yet sent to the broker. Wait up to close timeout for these pending requests to be processed. // If coordinator is not known, requests are aborted. Node coordinator = checkAndGetCoordinator(); if (coordinator != null && !client.awaitPendingRequests(coordinator, timer)) log.warn("Close timed out with {} pending requests to coordinator, terminating client connections", client.pendingRequestCount(coordinator)); } } }
@Test public void testNormalHeartbeat() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // normal heartbeat 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)); consumerClient.poll(time.timer(0)); assertTrue(future.isDone()); assertTrue(future.succeeded()); }
@Test public void blockWhenPollConditionNotSatisfied() { long timeout = 4000L; NetworkClient mockNetworkClient = mock(NetworkClient.class); ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(new LogContext(), mockNetworkClient, metadata, time, 100, 1000, Integer.MAX_VALUE); when(mockNetworkClient.inFlightRequestCount()).thenReturn(1); consumerClient.poll(time.timer(timeout), () -> true); verify(mockNetworkClient).poll(eq(timeout), anyLong()); }
consumerClient = new ConsumerNetworkClient(new LogContext(), client, metadata, time, 100, requestTimeoutMs, Integer.MAX_VALUE); RequestFuture<ClientResponse> future1 = consumerClient.send(node, heartbeat()); assertEquals(1, consumerClient.pendingRequestCount()); assertEquals(1, consumerClient.pendingRequestCount(node)); assertFalse(future1.isDone()); RequestFuture<ClientResponse> future2 = consumerClient.send(node, heartbeat()); assertEquals(2, consumerClient.pendingRequestCount()); assertEquals(2, consumerClient.pendingRequestCount(node)); assertFalse(future2.isDone()); consumerClient.poll(time.timer(0)); assertTrue(future1.isDone()); assertFalse(future1.succeeded()); assertEquals(1, consumerClient.pendingRequestCount()); assertEquals(1, consumerClient.pendingRequestCount(node)); assertFalse(future2.isDone()); consumerClient.poll(future2); ClientResponse clientResponse = future2.value(); HeartbeatResponse response = (HeartbeatResponse) clientResponse.responseBody(); RequestFuture<ClientResponse> future3 = consumerClient.send(node, heartbeat()); assertEquals(1, consumerClient.pendingRequestCount()); assertEquals(1, consumerClient.pendingRequestCount(node)); disconnected.set(true); consumerClient.poll(time.timer(0)); assertTrue(future3.isDone()); assertFalse(future3.succeeded());
@Test public void testDisconnectWithInFlightRequests() { RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat()); consumerClient.pollNoWakeup(); assertTrue(consumerClient.hasPendingRequests(node)); assertTrue(client.hasInFlightRequests(node.idString())); consumerClient.disconnectAsync(node); consumerClient.pollNoWakeup(); assertTrue(future.failed()); assertTrue(future.exception() instanceof DisconnectException); }
@Test public void wakeup() { RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat()); consumerClient.wakeup(); try { consumerClient.poll(time.timer(0)); fail(); } catch (WakeupException e) { } client.respond(heartbeatResponse(Errors.NONE)); consumerClient.poll(future); assertTrue(future.isDone()); }
@Override public void onSuccess(Void value) { pendingAsyncCommits.decrementAndGet(); doCommitOffsetsAsync(offsets, callback); client.pollNoWakeup(); }
client.poll(future, timer); if (future.isRetriable()) { log.debug("Coordinator discovery failed, refreshing metadata"); client.awaitMetadataUpdate(timer); } else throw future.exception(); } else if (coordinator != null && client.isUnavailable(coordinator)) {
@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()); }
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"); }
/** * Leave the current group and reset local generation/memberId. */ public synchronized void maybeLeaveGroup() { if (!coordinatorUnknown() && state != MemberState.UNJOINED && generation.isValid()) { // this is a minimal effort attempt to leave the group. we do not // attempt any resending if the request fails or times out. log.info("Sending LeaveGroup request to coordinator {}", coordinator); LeaveGroupRequest.Builder request = new LeaveGroupRequest.Builder(groupId, generation.memberId); client.send(coordinator, request) .compose(new LeaveGroupResponseHandler()); client.pollNoWakeup(); } resetGeneration(); }
private void setupCoordinator(int retryBackoffMs, int rebalanceTimeoutMs) { this.mockTime = new MockTime(); Metadata metadata = new Metadata(retryBackoffMs, 60 * 60 * 1000L, true); this.mockClient = new MockClient(mockTime, metadata); this.consumerClient = new ConsumerNetworkClient(new LogContext(), mockClient, metadata, mockTime, retryBackoffMs, REQUEST_TIMEOUT_MS, HEARTBEAT_INTERVAL_MS); Metrics metrics = new Metrics(); mockClient.updateMetadata(TestUtils.metadataUpdateWith(1, emptyMap())); this.node = metadata.fetch().nodes().get(0); this.coordinatorNode = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); this.coordinator = new DummyCoordinator(consumerClient, metrics, mockTime, rebalanceTimeoutMs, retryBackoffMs); }
@Test public void testTimeoutUnsentRequest() { // Delay connection to the node so that the request remains unsent client.delayReady(node, 1000); RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat(), 500); consumerClient.pollNoWakeup(); // Ensure the request is pending, but hasn't been sent assertTrue(consumerClient.hasPendingRequests()); assertFalse(client.hasInFlightRequests()); time.sleep(501); consumerClient.pollNoWakeup(); assertFalse(consumerClient.hasPendingRequests()); assertTrue(future.failed()); assertTrue(future.exception() instanceof TimeoutException); }
@Test public void multiSend() { client.prepareResponse(heartbeatResponse(Errors.NONE)); client.prepareResponse(heartbeatResponse(Errors.NONE)); RequestFuture<ClientResponse> future1 = consumerClient.send(node, heartbeat()); RequestFuture<ClientResponse> future2 = consumerClient.send(node, heartbeat()); assertEquals(2, consumerClient.pendingRequestCount()); assertEquals(2, consumerClient.pendingRequestCount(node)); consumerClient.awaitPendingRequests(node, time.timer(Long.MAX_VALUE)); assertTrue(future1.succeeded()); assertTrue(future2.succeeded()); }
@Test public void sendWithinBlackoutPeriodAfterAuthenticationFailure() throws InterruptedException { client.authenticationFailed(node, 300); client.prepareResponse(heartbeatResponse(Errors.NONE)); final RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat()); consumerClient.poll(future); assertTrue(future.failed()); assertTrue("Expected only an authentication error.", future.exception() instanceof AuthenticationException); time.sleep(30); // wait less than the blackout period assertTrue(client.connectionFailed(node)); final RequestFuture<ClientResponse> future2 = consumerClient.send(node, heartbeat()); consumerClient.poll(future2); assertTrue(future2.failed()); assertTrue("Expected only an authentication error.", future2.exception() instanceof AuthenticationException); }