diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java index 8e6f3d7baac2..635b8be07048 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/TaskInfo.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.runtime.rest.entities; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -26,7 +27,8 @@ public class TaskInfo { private final ConnectorTaskId id; private final Map config; - public TaskInfo(ConnectorTaskId id, Map config) { + @JsonCreator + public TaskInfo(@JsonProperty("id") ConnectorTaskId id, @JsonProperty("config") Map config) { this.id = id; this.config = config; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index 9f1cabcfa14a..4878b8df9e1f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -173,9 +173,11 @@ public Map getConnectorConfig(final @PathParam("connector") Stri @GET @Path("/{connector}/tasks-config") - @Operation(summary = "Get the configuration of all tasks for the specified connector") + @Operation(deprecated = true, summary = "Get the configuration of all tasks for the specified connector") public Map> getTasksConfig( final @PathParam("connector") String connector) throws Throwable { + log.warn("The 'GET /connectors/{connector}/tasks-config' endpoint is deprecated and will be removed in the next major release. " + + "Please use the 'GET /connectors/{connector}/tasks' endpoint instead."); FutureCallback>> cb = new FutureCallback<>(); herder.tasksConfig(connector, cb); return requestHandler.completeRequest(cb); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index 4c393d95ad36..42c3831faf49 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.connect.integration; +import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.runtime.rest.resources.ConnectorsResource; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.WorkerHandle; @@ -29,9 +31,11 @@ import org.junit.rules.TestRule; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.event.Level; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Properties; @@ -51,6 +55,8 @@ import static org.apache.kafka.connect.runtime.WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; import static org.apache.kafka.connect.util.clusters.EmbeddedConnectClusterAssertions.CONNECTOR_SETUP_DURATION_MS; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -382,7 +388,7 @@ public void testPauseStopResume() throws Exception { ); // If the connector is truly stopped, we should also see an empty set of tasks and task configs assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks()); - assertEquals(Collections.emptyMap(), connect.taskConfigs(CONNECTOR_NAME)); + assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME)); // Transition to RUNNING connect.resumeConnector(CONNECTOR_NAME); @@ -411,7 +417,7 @@ public void testPauseStopResume() throws Exception { "Connector did not stop in time" ); assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks()); - assertEquals(Collections.emptyMap(), connect.taskConfigs(CONNECTOR_NAME)); + assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME)); // Transition to PAUSED connect.pauseConnector(CONNECTOR_NAME); @@ -471,7 +477,7 @@ public void testStoppedState() throws Exception { ); // If the connector is truly stopped, we should also see an empty set of tasks and task configs assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks()); - assertEquals(Collections.emptyMap(), connect.taskConfigs(CONNECTOR_NAME)); + assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME)); // Can resume a connector after its Connector has failed before shutdown after receiving a stop request props.remove("connector.start.inject.error"); @@ -493,7 +499,7 @@ public void testStoppedState() throws Exception { "Connector did not stop in time" ); assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks()); - assertEquals(Collections.emptyMap(), connect.taskConfigs(CONNECTOR_NAME)); + assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME)); // Can resume a connector after its Connector has failed during shutdown after receiving a stop request connect.resumeConnector(CONNECTOR_NAME); @@ -511,6 +517,37 @@ public void testStoppedState() throws Exception { ); } + /** + * The GET /connectors/{connector}/tasks-config endpoint was deprecated in + * KIP-970 + * and is slated for removal in the next major release. This test verifies that the deprecation warning log is emitted on trying to use the + * deprecated endpoint. + */ + @Test + public void testTasksConfigDeprecation() throws Exception { + connect = connectBuilder.build(); + // start the clusters + connect.start(); + + connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, + "Initial group of workers did not start in time."); + + connect.configureConnector(CONNECTOR_NAME, defaultSourceConnectorProps(TOPIC_NAME)); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning( + CONNECTOR_NAME, + NUM_TASKS, + "Connector tasks did not start in time" + ); + + try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(ConnectorsResource.class)) { + connect.requestGet(connect.endpointForResource("connectors/" + CONNECTOR_NAME + "/tasks-config")); + List logEvents = logCaptureAppender.getEvents(); + assertEquals(1, logEvents.size()); + assertEquals(Level.WARN.toString(), logEvents.get(0).getLevel()); + assertThat(logEvents.get(0).getMessage(), containsString("deprecated")); + } + } + private Map defaultSourceConnectorProps(String topic) { // setup up props for the source connector Map props = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java index 20dce332fcfa..9e3edcd8f242 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java @@ -31,6 +31,7 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.ServerInfo; +import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; import org.apache.kafka.connect.util.SinkUtils; import org.eclipse.jetty.client.HttpClient; @@ -611,19 +612,19 @@ public ConnectorInfo connectorInfo(String connectorName) { /** * Get the task configs of a connector running in this cluster. - + * * @param connectorName name of the connector - * @return a map from task ID (connector name + "-" + task number) to task config + * @return a list of task configurations for the connector */ - public Map> taskConfigs(String connectorName) { + public List taskConfigs(String connectorName) { ObjectMapper mapper = new ObjectMapper(); - String url = endpointForResource(String.format("connectors/%s/tasks-config", connectorName)); + String url = endpointForResource(String.format("connectors/%s/tasks", connectorName)); Response response = requestGet(url); try { if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { // We use String instead of ConnectorTaskId as the key here since the latter can't be automatically // deserialized by Jackson when used as a JSON object key (i.e., when it's serialized as a JSON string) - return mapper.readValue(responseToString(response), new TypeReference>>() { }); + return mapper.readValue(responseToString(response), new TypeReference>() { }); } } catch (IOException e) { log.error("Could not read task configs from response: {}", diff --git a/docs/connect.html b/docs/connect.html index 2deb89018882..16f268e78b0a 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -298,7 +298,8 @@

REST API

  • GET /connectors/{name}/config - get the configuration parameters for a specific connector
  • PUT /connectors/{name}/config - update the configuration parameters for a specific connector
  • GET /connectors/{name}/status - get current status of the connector, including if it is running, failed, paused, etc., which worker it is assigned to, error information if it has failed, and the state of all its tasks
  • -
  • GET /connectors/{name}/tasks - get a list of tasks currently running for a connector
  • +
  • GET /connectors/{name}/tasks - get a list of tasks currently running for a connector along with their configurations
  • +
  • GET /connectors/{name}/tasks-config - get the configuration of all tasks for a specific connector. This endpoint is deprecated and will be removed in the next major release. Please use the GET /connectors/{name}/tasks endpoint instead. Note that the response structures of the two endpoints differ slightly, please refer to the OpenAPI documentation for more details
  • GET /connectors/{name}/tasks/{taskid}/status - get current status of the task, including if it is running, failed, paused, etc., which worker it is assigned to, and error information if it has failed
  • PUT /connectors/{name}/pause - pause the connector and its tasks, which stops message processing until the connector is resumed. Any resources claimed by its tasks are left allocated, which allows the connector to begin processing data quickly once it is resumed.
  • PUT /connectors/{name}/stop - stop the connector and shut down its tasks, deallocating any resources claimed by its tasks. This is more efficient from a resource usage standpoint than pausing the connector, but can cause it to take longer to begin processing data once resumed. Note that the offsets for a connector can be only modified via the offsets management endpoints if it is in the stopped state