kpatelatwork commented on a change in pull request #10822: URL: https://github.com/apache/kafka/pull/10822#discussion_r655749684
########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java ########## @@ -1063,6 +1076,132 @@ public int generation() { return generation; } + @Override + public void restartConnectorAndTasks( + RestartRequest request, + Callback<ConnectorStateInfo> callback + ) { + final String connectorName = request.connectorName(); + addRequest( + () -> { + if (checkRebalanceNeeded(callback)) { + return null; + } + if (!configState.connectors().contains(request.connectorName())) { + callback.onCompletion(new NotFoundException("Unknown connector: " + connectorName), null); + return null; + } + if (isLeader()) { + // Write a restart request to the config backing store, to be executed asynchronously in tick() + configBackingStore.putRestartRequest(request); + // Compute and send the response that this was accepted + Optional<RestartPlan> maybePlan = buildRestartPlanFor(request); + if (!maybePlan.isPresent()) { + callback.onCompletion(new NotFoundException("Status for connector " + connectorName + " not found", null), null); + } else { + RestartPlan plan = maybePlan.get(); + callback.onCompletion(null, plan.restartConnectorStateInfo()); + } + } else { + callback.onCompletion(new NotLeaderException("Only the leader can process restart requests.", leaderUrl()), null); + } + + return null; + }, + forwardErrorCallback(callback) + ); + } + + /** + * Process all pending restart requests. There can be at most one request per connector. + * + * <p>This method is called from within the {@link #tick()} method. + */ + void processRestartRequests() { + List<RestartRequest> restartRequests; + synchronized (this) { + if (pendingRestartRequests.isEmpty()) { + return; + } + //dequeue into a local list to minimize the work being done within the synchronized block + restartRequests = new ArrayList<>(pendingRestartRequests.values()); + pendingRestartRequests.clear(); + } + for (RestartRequest restartRequest : restartRequests) { + try { + doRestartConnectorAndTasks(restartRequest); + } catch (Exception e) { + log.warn("Unexpected error while trying to process " + restartRequest + ", the restart request will be skipped.", e); + } + } + } + + /** + * Builds and and executes a restart plan for the connector and its tasks from <code>request</code>. + * Execution of a plan involves triggering the stop of eligible connector/tasks and then queuing the start for eligible connector/tasks. + * + * @param request the request to restart connector and tasks + */ + protected synchronized void doRestartConnectorAndTasks(RestartRequest request) { + final String connectorName = request.connectorName(); + Optional<RestartPlan> maybePlan = buildRestartPlanFor(request); + if (!maybePlan.isPresent()) { + log.debug("Skipping restart of connector '{}' since no status is available: {}", connectorName, request); + return; + } + RestartPlan plan = maybePlan.get(); + log.info("Executing {}", plan); + + // If requested, stop the connector and any tasks, marking each as restarting + final ExtendedAssignment currentAssignments = assignment; + final Collection<ConnectorTaskId> assignedIdsToRestart = plan.taskIdsToRestart() + .stream() + .filter(taskId -> currentAssignments.tasks().contains(taskId)) + .collect(Collectors.toList()); + final boolean restartConnector = plan.restartConnector() && currentAssignments.connectors().contains(connectorName); + final boolean restartTasks = !assignedIdsToRestart.isEmpty(); + if (restartConnector) { + worker.stopAndAwaitConnector(connectorName); + recordRestarting(connectorName); + } + if (restartTasks) { + // Stop the tasks and mark as restarting + worker.stopAndAwaitTasks(assignedIdsToRestart); + assignedIdsToRestart.forEach(this::recordRestarting); + } + + // Now restart the connector and tasks + if (restartConnector) { + try { + startConnector(connectorName, (error, targetState) -> { + if (error == null) { + log.info("Connector '{}' restart successful", connectorName); + } else { + log.error("Connector '{}' restart failed", connectorName, error); + } + }); + } catch (Throwable t) { + log.error("Connector '{}' restart failed", connectorName, t); + } Review comment: @rhauch I kinda agree with you but I had seen this code and it had try/catch around both startTask and startConnector . I added try/catch around startTask so if one tasks fails we can see if others atleasst succeed. The try/catch around startConnector was because I saw the pattern. But I am torn on this, if you think we should remove the try/catch on connector, I am open to removing the try/catch given its already caught at the higher layer, but IMHO we can keep the startTask try/catch. Wdyt? https://github.com/apache/kafka/blob/1bd99c809b6546cd6711cc4f2d40785393ca584a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L1437 https://github.com/apache/kafka/blob/1bd99c809b6546cd6711cc4f2d40785393ca584a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java#L1490 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org