-
Notifications
You must be signed in to change notification settings - Fork 14.9k
KAFKA-13780: Generate OpenAPI file for Connect REST API #12067
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,8 @@ | |
| */ | ||
| package org.apache.kafka.connect.runtime.rest.resources; | ||
|
|
||
| import io.swagger.v3.oas.annotations.Operation; | ||
| import io.swagger.v3.oas.annotations.Parameter; | ||
| import org.apache.kafka.connect.runtime.ConnectorConfig; | ||
| import org.apache.kafka.connect.runtime.Herder; | ||
| import org.apache.kafka.connect.runtime.PredicatedTransformation; | ||
|
|
@@ -102,17 +104,18 @@ private <T> void addConnectorPlugins(Collection<PluginDesc<T>> plugins, Collecti | |
| } | ||
|
|
||
| @PUT | ||
| @Path("/{connectorType}/config/validate") | ||
| @Path("/{pluginName}/config/validate") | ||
| @Operation(summary = "Validate the provided configuration against the configuration definition for the specified pluginName") | ||
| public ConfigInfos validateConfigs( | ||
| final @PathParam("connectorType") String connType, | ||
| final @PathParam("pluginName") String pluginName, | ||
| final Map<String, String> connectorConfig | ||
| ) throws Throwable { | ||
| String includedConnType = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG); | ||
| if (includedConnType != null | ||
| && !normalizedPluginName(includedConnType).endsWith(normalizedPluginName(connType))) { | ||
| && !normalizedPluginName(includedConnType).endsWith(normalizedPluginName(pluginName))) { | ||
| throw new BadRequestException( | ||
| "Included connector type " + includedConnType + " does not match request type " | ||
| + connType | ||
| + pluginName | ||
| ); | ||
| } | ||
|
|
||
|
|
@@ -133,7 +136,10 @@ public ConfigInfos validateConfigs( | |
|
|
||
| @GET | ||
| @Path("/") | ||
| public List<PluginInfo> listConnectorPlugins(@DefaultValue("true") @QueryParam("connectorsOnly") boolean connectorsOnly) { | ||
| @Operation(summary = "List all connector plugins installed") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should explain the behaivour of the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done |
||
| public List<PluginInfo> listConnectorPlugins( | ||
| @DefaultValue("true") @QueryParam("connectorsOnly") @Parameter(description = "Whether to list only connectors instead of all plugins") boolean connectorsOnly | ||
| ) { | ||
| synchronized (this) { | ||
| if (connectorsOnly) { | ||
| return Collections.unmodifiableList(connectorPlugins.stream() | ||
|
|
@@ -146,8 +152,9 @@ public List<PluginInfo> listConnectorPlugins(@DefaultValue("true") @QueryParam(" | |
| } | ||
|
|
||
| @GET | ||
| @Path("/{name}/config") | ||
| public List<ConfigKeyInfo> getConnectorConfigDef(final @PathParam("name") String pluginName) { | ||
| @Path("/{pluginName}/config") | ||
| @Operation(summary = "Get the configuration definition for the specified pluginName") | ||
| public List<ConfigKeyInfo> getConnectorConfigDef(final @PathParam("pluginName") String pluginName) { | ||
| synchronized (this) { | ||
| return herder.connectorPluginConfig(pluginName); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,8 @@ | |
| import javax.ws.rs.core.HttpHeaders; | ||
|
|
||
| import com.fasterxml.jackson.databind.ObjectMapper; | ||
| import io.swagger.v3.oas.annotations.Operation; | ||
| import io.swagger.v3.oas.annotations.Parameter; | ||
| import org.apache.kafka.connect.errors.NotFoundException; | ||
| import org.apache.kafka.connect.runtime.ConnectorConfig; | ||
| import org.apache.kafka.connect.runtime.Herder; | ||
|
|
@@ -113,6 +115,7 @@ public static void resetRequestTimeout() { | |
|
|
||
| @GET | ||
| @Path("/") | ||
| @Operation(summary = "List all active connectors") | ||
| public Response listConnectors( | ||
| final @Context UriInfo uriInfo, | ||
| final @Context HttpHeaders headers | ||
|
|
@@ -150,7 +153,8 @@ public Response listConnectors( | |
|
|
||
| @POST | ||
| @Path("/") | ||
| public Response createConnector(final @QueryParam("forward") Boolean forward, | ||
| @Operation(summary = "Create a new connector") | ||
| public Response createConnector(final @Parameter(hidden = true) @QueryParam("forward") Boolean forward, | ||
| final @Context HttpHeaders headers, | ||
| final CreateConnectorRequest createRequest) throws Throwable { | ||
| // Trim leading and trailing whitespaces from the connector name, replace null with empty string | ||
|
|
@@ -172,43 +176,48 @@ public Response createConnector(final @QueryParam("forward") Boolean forward, | |
|
|
||
| @GET | ||
| @Path("/{connector}") | ||
| @Operation(summary = "Get the details for the specified connector") | ||
| public ConnectorInfo getConnector(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward) throws Throwable { | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable { | ||
| FutureCallback<ConnectorInfo> cb = new FutureCallback<>(); | ||
| herder.connectorInfo(connector, cb); | ||
| return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", headers, null, forward); | ||
| } | ||
|
|
||
| @GET | ||
| @Path("/{connector}/config") | ||
| @Operation(summary = "Get the configuration for the specified connector") | ||
| public Map<String, String> getConnectorConfig(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward) throws Throwable { | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable { | ||
| FutureCallback<Map<String, String>> cb = new FutureCallback<>(); | ||
| herder.connectorConfig(connector, cb); | ||
| return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", headers, null, forward); | ||
| } | ||
|
|
||
| @GET | ||
| @Path("/{connector}/tasks-config") | ||
| @Operation(summary = "Get the configuration of all tasks for the specified connector") | ||
| public Map<ConnectorTaskId, Map<String, String>> getTasksConfig( | ||
| final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward) throws Throwable { | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable { | ||
| FutureCallback<Map<ConnectorTaskId, Map<String, String>>> cb = new FutureCallback<>(); | ||
| herder.tasksConfig(connector, cb); | ||
| return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks-config", "GET", headers, null, forward); | ||
| } | ||
|
|
||
| @GET | ||
| @Path("/{connector}/status") | ||
| @Operation(summary = "Get the status for the specified connector") | ||
| public ConnectorStateInfo getConnectorStatus(final @PathParam("connector") String connector) { | ||
| return herder.connectorStatus(connector); | ||
| } | ||
|
|
||
| @GET | ||
| @Path("/{connector}/topics") | ||
| @Operation(summary = "Get the list of topics actively used by the specified connector") | ||
| public Response getConnectorActiveTopics(final @PathParam("connector") String connector) { | ||
| if (isTopicTrackingDisabled) { | ||
| throw new ConnectRestException(Response.Status.FORBIDDEN.getStatusCode(), | ||
|
|
@@ -220,6 +229,7 @@ public Response getConnectorActiveTopics(final @PathParam("connector") String co | |
|
|
||
| @PUT | ||
| @Path("/{connector}/topics/reset") | ||
| @Operation(summary = "Reset the list of topics actively used by the specified connector") | ||
| public Response resetConnectorActiveTopics(final @PathParam("connector") String connector, final @Context HttpHeaders headers) { | ||
| if (isTopicTrackingDisabled) { | ||
| throw new ConnectRestException(Response.Status.FORBIDDEN.getStatusCode(), | ||
|
|
@@ -235,9 +245,10 @@ public Response resetConnectorActiveTopics(final @PathParam("connector") String | |
|
|
||
| @PUT | ||
| @Path("/{connector}/config") | ||
| @Operation(summary = "Create or reconfigure the specified connector") | ||
| public Response putConnectorConfig(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward, | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward, | ||
| final Map<String, String> connectorConfig) throws Throwable { | ||
| FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>(); | ||
| checkAndPutConnectorConfigName(connector, connectorConfig); | ||
|
|
@@ -257,11 +268,12 @@ public Response putConnectorConfig(final @PathParam("connector") String connecto | |
|
|
||
| @POST | ||
| @Path("/{connector}/restart") | ||
| @Operation(summary = "Restart the specified connector") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we mention
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I've added descriptions for these parameters |
||
| public Response restartConnector(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @DefaultValue("false") @QueryParam("includeTasks") Boolean includeTasks, | ||
| final @DefaultValue("false") @QueryParam("onlyFailed") Boolean onlyFailed, | ||
| final @QueryParam("forward") Boolean forward) throws Throwable { | ||
| final @DefaultValue("false") @QueryParam("includeTasks") @Parameter(description = "Whether to also restart tasks") Boolean includeTasks, | ||
| final @DefaultValue("false") @QueryParam("onlyFailed") @Parameter(description = "Whether to only restart failed tasks/connectors")Boolean onlyFailed, | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable { | ||
| RestartRequest restartRequest = new RestartRequest(connector, onlyFailed, includeTasks); | ||
| String forwardingPath = "/connectors/" + connector + "/restart"; | ||
| if (restartRequest.forceRestartConnectorOnly()) { | ||
|
|
@@ -285,23 +297,28 @@ public Response restartConnector(final @PathParam("connector") String connector, | |
|
|
||
| @PUT | ||
| @Path("/{connector}/pause") | ||
| @Operation(summary = "Pause the specified connector", | ||
| description = "This operation is idempotent and has no effects if the connector is already paused") | ||
| public Response pauseConnector(@PathParam("connector") String connector, final @Context HttpHeaders headers) { | ||
| herder.pauseConnector(connector); | ||
| return Response.accepted().build(); | ||
| } | ||
|
|
||
| @PUT | ||
| @Path("/{connector}/resume") | ||
| @Operation(summary = "Resume the specified connector", | ||
| description = "This operation is idempotent and has no effects if the connector is already running") | ||
| public Response resumeConnector(@PathParam("connector") String connector) { | ||
| herder.resumeConnector(connector); | ||
| return Response.accepted().build(); | ||
| } | ||
|
|
||
| @GET | ||
| @Path("/{connector}/tasks") | ||
| @Operation(summary = "List all tasks for the specified connector") | ||
| public List<TaskInfo> getTaskConfigs(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward) throws Throwable { | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable { | ||
| FutureCallback<List<TaskInfo>> cb = new FutureCallback<>(); | ||
| herder.taskConfigs(connector, cb); | ||
| return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", headers, null, new TypeReference<List<TaskInfo>>() { | ||
|
|
@@ -310,6 +327,7 @@ public List<TaskInfo> getTaskConfigs(final @PathParam("connector") String connec | |
|
|
||
| @POST | ||
| @Path("/{connector}/tasks") | ||
| @Operation(hidden = true, summary = "This operation is only for inter-worker communications") | ||
| public void putTaskConfigs(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward, | ||
|
|
@@ -322,6 +340,7 @@ public void putTaskConfigs(final @PathParam("connector") String connector, | |
|
|
||
| @GET | ||
| @Path("/{connector}/tasks/{task}/status") | ||
| @Operation(summary = "Get the state of the specified task for the specified connector") | ||
| public ConnectorStateInfo.TaskState getTaskStatus(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @PathParam("task") Integer task) { | ||
|
|
@@ -330,10 +349,11 @@ public ConnectorStateInfo.TaskState getTaskStatus(final @PathParam("connector") | |
|
|
||
| @POST | ||
| @Path("/{connector}/tasks/{task}/restart") | ||
| @Operation(summary = "Restart the specified task for the specified connector") | ||
| public void restartTask(final @PathParam("connector") String connector, | ||
| final @PathParam("task") Integer task, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward) throws Throwable { | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable { | ||
| FutureCallback<Void> cb = new FutureCallback<>(); | ||
| ConnectorTaskId taskId = new ConnectorTaskId(connector, task); | ||
| herder.restartTask(taskId, cb); | ||
|
|
@@ -342,9 +362,10 @@ public void restartTask(final @PathParam("connector") String connector, | |
|
|
||
| @DELETE | ||
| @Path("/{connector}") | ||
| @Operation(summary = "Delete the specified connector") | ||
| public void destroyConnector(final @PathParam("connector") String connector, | ||
| final @Context HttpHeaders headers, | ||
| final @QueryParam("forward") Boolean forward) throws Throwable { | ||
| final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable { | ||
| FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>(); | ||
| herder.deleteConnectorConfig(connector, cb); | ||
| completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", headers, null, forward); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,7 @@ | |
| */ | ||
| package org.apache.kafka.connect.runtime.rest.resources; | ||
|
|
||
| import io.swagger.v3.oas.annotations.Operation; | ||
| import org.apache.kafka.connect.errors.NotFoundException; | ||
| import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; | ||
| import org.apache.log4j.Level; | ||
|
|
@@ -59,6 +60,7 @@ public class LoggingResource { | |
| */ | ||
| @GET | ||
| @Path("/") | ||
| @Operation(summary = "List the current loggers that have their levels explicitly set and their log levels") | ||
| public Response listLoggers() { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes this is not great. I think this should be fixable without requiring a KIP, but let's do this in a separate PR |
||
| Map<String, Map<String, String>> loggers = new TreeMap<>(); | ||
| Enumeration<Logger> enumeration = currentLoggers(); | ||
|
|
@@ -83,6 +85,7 @@ public Response listLoggers() { | |
| */ | ||
| @GET | ||
| @Path("/{logger}") | ||
| @Operation(summary = "Get the log level for the specified logger") | ||
| public Response getLogger(final @PathParam("logger") String namedLogger) { | ||
| Objects.requireNonNull(namedLogger, "require non-null name"); | ||
|
|
||
|
|
@@ -120,6 +123,7 @@ public Response getLogger(final @PathParam("logger") String namedLogger) { | |
| */ | ||
| @PUT | ||
| @Path("/{logger}") | ||
| @Operation(summary = "Set the level for the specified logger") | ||
| public Response setLevel(final @PathParam("logger") String namedLogger, | ||
| final Map<String, String> levelMap) { | ||
| String desiredLevelStr = levelMap.get("level"); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It also depends on having the connect runtime classes/jar.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I believe this happens automatically because of
classpath = sourceSets.main.runtimeClasspath