/** * Close the coordinator, waiting if needed to send LeaveGroup. */ @Override public final void close() { close(time.timer(0)); }
/** * Ensure the group is active (i.e., joined and synced) * * @param timer Timer bounding how long this method can block * @return true iff the group is active */ boolean ensureActiveGroup(final Timer timer) { // always ensure that the coordinator is ready because we may have been disconnected // when sending heartbeats and does not necessarily require us to rejoin the group. if (!ensureCoordinatorReady(timer)) { return false; } startHeartbeatThreadIfNeeded(); return joinGroupIfNeeded(timer); }
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)); } } }
disableHeartbeatThread(); joinFuture = sendJoinGroupRequest(); joinFuture.addListener(new RequestFutureListener<ByteBuffer>() { @Override
while (rejoinNeededOrPending()) { if (!ensureCoordinatorReady(timer)) { return false; onJoinPrepare(generation.generationId, generation.memberId); needsJoinPrepare = false; final RequestFuture<ByteBuffer> future = initiateJoinGroup(); client.poll(future, timer); if (!future.isDone()) { onJoinComplete(generation.generationId, generation.memberId, generation.protocol, memberAssignment); resetJoinGroupFuture(); needsJoinPrepare = true; } else { resetJoinGroupFuture(); final RuntimeException exception = future.exception(); if (exception instanceof UnknownMemberIdException ||
long now = time.milliseconds(); if (coordinatorUnknown()) { if (findCoordinatorFuture != null || lookupCoordinator().failed()) markCoordinatorUnknown(); } else if (heartbeat.pollTimeoutExpired(now)) { "max.poll.interval.ms or by reducing the maximum size of batches returned in poll() " + "with max.poll.records."); maybeLeaveGroup(); } else if (!heartbeat.shouldHeartbeat(now)) { heartbeat.sentHeartbeat(now); sendHeartbeatRequest().addListener(new RequestFutureListener<Void>() { @Override public void onSuccess(Void value) {
/** * Join the group and return the assignment for the next generation. This function handles both * JoinGroup and SyncGroup, delegating to {@link #performAssignment(String, String, Map)} if * elected leader by the coordinator. * * NOTE: This is visible only for testing * * @return A request future which wraps the assignment returned from the group leader */ RequestFuture<ByteBuffer> sendJoinGroupRequest() { if (coordinatorUnknown()) return RequestFuture.coordinatorNotAvailable(); // send a join group request to the coordinator log.info("(Re-)joining group"); JoinGroupRequest.Builder requestBuilder = new JoinGroupRequest.Builder( groupId, this.sessionTimeoutMs, this.generation.memberId, protocolType(), metadata()).setRebalanceTimeout(this.rebalanceTimeoutMs); log.debug("Sending JoinGroup ({}) to coordinator {}", requestBuilder, this.coordinator); // Note that we override the request timeout using the rebalance timeout since that is the // maximum time that it may block on the coordinator. We add an extra 5 seconds for small delays. int joinGroupTimeoutMs = Math.max(rebalanceTimeoutMs, rebalanceTimeoutMs + 5000); return client.send(coordinator, requestBuilder, joinGroupTimeoutMs) .compose(new JoinGroupResponseHandler()); }
if (!coordinatorUnknown()) return true; final RequestFuture<Void> future = lookupCoordinator(); client.poll(future, timer); markCoordinatorUnknown(); timer.sleep(retryBackoffMs); } while (coordinatorUnknown() && timer.notExpired()); return !coordinatorUnknown();
@Override public boolean rejoinNeededOrPending() { if (!subscriptions.partitionsAutoAssigned()) return false; // we need to rejoin if we performed the assignment and metadata has changed if (assignmentSnapshot != null && !assignmentSnapshot.equals(metadataSnapshot)) return true; // we need to join if our subscription has changed since the last join if (joinedSubscription != null && !joinedSubscription.equals(subscriptions.subscription())) return true; return super.rejoinNeededOrPending(); }
/** * 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(); }
@Override protected synchronized boolean ensureCoordinatorReady(final Timer timer) { return super.ensureCoordinatorReady(timer); }
/** * Check if we know who the coordinator is and we have an active connection * @return true if the coordinator is unknown */ public boolean coordinatorUnknown() { return checkAndGetCoordinator() == null; }
/** * Ensure that the group is active (i.e. joined and synced) */ public void ensureActiveGroup() { while (!ensureActiveGroup(time.timer(Long.MAX_VALUE))) { log.warn("still waiting to ensure active group"); } }
private RequestFuture<ByteBuffer> sendSyncGroupRequest(SyncGroupRequest.Builder requestBuilder) { if (coordinatorUnknown()) return RequestFuture.coordinatorNotAvailable(); return client.send(coordinator, requestBuilder) .compose(new SyncGroupResponseHandler()); }
this.metricGrpName, "The max time taken to receive a response to a heartbeat request"), new Max()); this.heartbeatLatency.add(createMeter(metrics, metricGrpName, "heartbeat", "heartbeats")); this.metricGrpName, "The max time taken for a group rejoin"), new Max()); this.joinLatency.add(createMeter(metrics, metricGrpName, "join", "group joins")); this.metricGrpName, "The max time taken for a group sync"), new Max()); this.syncLatency.add(createMeter(metrics, metricGrpName, "sync", "group syncs"));
@Override protected boolean rejoinNeededOrPending() { return super.rejoinNeededOrPending() || (assignmentSnapshot == null || assignmentSnapshot.failed()) || rejoinRequested; }
public void close(final Timer timer) { // we do not need to re-enable wakeups since we are closing already client.disableWakeups(); try { maybeAutoCommitOffsetsSync(timer); while (pendingAsyncCommits.get() > 0 && timer.notExpired()) { ensureCoordinatorReady(timer); client.poll(timer); invokeCompletedOffsetCommitCallbacks(); } } finally { super.close(timer); } }
@Override public void close() { super.close(); }