kpatelatwork commented on a change in pull request #10822: URL: https://github.com/apache/kafka/pull/10822#discussion_r655408049
########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java ########## @@ -1063,6 +1076,112 @@ 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); + } + } + } + + 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) { + startConnector(connectorName, (error, targetState) -> { + if (error == null) { + log.info("Connector '{}' successfully restarted", connectorName); + } else { + log.error("Failed to restart connector '" + connectorName + "'", error); + } + }); + } + if (restartTasks) { + log.debug("Restarting {} of {} tasks for {}", plan.restartTaskCount(), plan.totalTaskCount(), request); + plan.taskIdsToRestart().forEach(this::startTask); + log.debug("Restarted {} of {} tasks for {} as requested", plan.restartTaskCount(), plan.totalTaskCount(), request); + } Review comment: @rhauch we can add a retryCount in the RestartRequest and start with a default of 5 and decrement it every time we re-enqueue. If the tick() happens every second then we can exhaust the retries pretty fast so we may need to add backoff with a delay before processing the request. Adding retry logic to this PR can make the PR complex, I would vote for adding it in a separate PR. ########## 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 ########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java ########## @@ -1592,6 +1731,28 @@ public void onSessionKeyUpdate(SessionKey sessionKey) { } } } + + @Override + public void onRestartRequest(RestartRequest request) { + log.info("Received and enqueuing {}", request); + + synchronized (DistributedHerder.this) { + String connectorName = request.connectorName(); + //preserve the highest impact request + if (pendingRestartRequests.containsKey(connectorName)) { + RestartRequest existingRequest = pendingRestartRequests.get(connectorName); + if (request.compareTo(existingRequest) > 0) { + log.debug("Overwriting existing {} and enqueuing the higher impact {}", existingRequest, request); + pendingRestartRequests.put(connectorName, request); + } else { + log.debug("Preserving existing higher impact {} and ignoring incoming {}", existingRequest, request); + } + } else { + pendingRestartRequests.put(connectorName, request); + } Review comment: turns out a normal hashmap also has compute so I just used it. ########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/RestartRequest.java ########## @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.runtime; + +import java.util.Objects; + +import org.apache.kafka.connect.connector.Connector; +import org.apache.kafka.connect.connector.Task; + +/** + * A request to restart a connector and/or task instances. Review comment: @rhauch I added some Javadoc, could you please review and see if it looks good? ########## File path: connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java ########## @@ -559,6 +560,31 @@ public String endpointForResource(String resource) { return url + resource; } + /** + * Get the full URL of the endpoint that corresponds to the given REST resource using a worker + * that is not running any tasks or connector instance for the connectorName provided in the arguments + * + * @param resource the resource under the worker's admin endpoint + * @param connectorName the name of the connector + * @return the admin endpoint URL + * @throws ConnectException if no REST endpoint is available + */ + public String endpointForWorkerRunningNoResourceForConnector(String resource, String connectorName) { Review comment: @rhauch I think it's a good idea but I saw that org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster#endpointForResource method is using the same pattern and if I change it then I will have to change a lot of classes not related to this PR and ultimately in the test I will have to check for Optional.present and do an assert. The situation for not finding a worker is rare and doing empty check and assert in all places will be a lot of copy/paste whereas throwing an exception here is centralizing this rare situation. Do you think it's ok to leave it as is? ########## 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 in below two links 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