From 4fa81ca37b3e17ceecf6d0490367a9d580d6990d Mon Sep 17 00:00:00 2001 From: ilkerkocatepe Date: Sun, 13 Oct 2024 17:23:27 +0300 Subject: [PATCH] [#434] fix: after review - partitions and replication_factor parameters set optional Signed-off-by: ilkerkocatepe --- .../openapi.json-generate-apidoc.sha256 | 2 +- documentation/book/api/index.adoc | 14 +++++++------- .../strimzi/kafka/bridge/KafkaBridgeAdmin.java | 8 ++------ .../bridge/http/HttpAdminBridgeEndpoint.java | 12 +++++------- src/main/resources/openapi.json | 16 ++++++++-------- src/main/resources/openapiv2.json | 14 +++++++------- .../strimzi/kafka/bridge/http/AdminClientIT.java | 14 ++++++++++++++ 7 files changed, 44 insertions(+), 36 deletions(-) diff --git a/documentation/book/api/.openapi-generator/openapi.json-generate-apidoc.sha256 b/documentation/book/api/.openapi-generator/openapi.json-generate-apidoc.sha256 index 8af8177f..0facbd30 100644 --- a/documentation/book/api/.openapi-generator/openapi.json-generate-apidoc.sha256 +++ b/documentation/book/api/.openapi-generator/openapi.json-generate-apidoc.sha256 @@ -1 +1 @@ -88546893f4a6b52b04e3729153f4faea9356730c8facc0930093aa956530d184 \ No newline at end of file +bf9a3ec3619b7fe86b6473ec93bb17c589dff7935eacd41636ea071274110d55 \ No newline at end of file diff --git a/documentation/book/api/index.adoc b/documentation/book/api/index.adoc index 609d4ccd..923466e6 100644 --- a/documentation/book/api/index.adoc +++ b/documentation/book/api/index.adoc @@ -2314,7 +2314,7 @@ Creates a topic with given topic name. |Name| Description| Required| Default| Pattern | topicname -| Name of the topic to send records to or retrieve metadata from. +| Name of the topic will be created. | X | null | @@ -2331,14 +2331,14 @@ Creates a topic with given topic name. |Name| Description| Required| Default| Pattern | partitions -| Number of partitions for the topic. -| X +| Number of partitions for the topic. (Optional) +| - | null | | replication_factor -| Replication factor for the topic. -| X +| Replication factor for the topic. (Optional) +| - | null | @@ -2369,7 +2369,7 @@ Creates a topic with given topic name. | 404 -| The specified topic was not found. +| The path was not found. | <> |=== @@ -2645,7 +2645,7 @@ Retrieves the metadata about a given topic. | 404 -| The specified topic was not found. +| The path topic was not found. | <> |=== diff --git a/src/main/java/io/strimzi/kafka/bridge/KafkaBridgeAdmin.java b/src/main/java/io/strimzi/kafka/bridge/KafkaBridgeAdmin.java index fe8db844..0fd29fac 100644 --- a/src/main/java/io/strimzi/kafka/bridge/KafkaBridgeAdmin.java +++ b/src/main/java/io/strimzi/kafka/bridge/KafkaBridgeAdmin.java @@ -17,11 +17,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; +import java.util.*; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; @@ -94,7 +90,7 @@ public CompletionStage> listTopics() { * @param replicationFactor replication factor * @return a CompletionStage Void */ - public CompletionStage createTopic(String topicName, int partitions, short replicationFactor) { + public CompletionStage createTopic(String topicName, Optional partitions, Optional replicationFactor) { LOGGER.trace("Create topic thread {}", Thread.currentThread()); LOGGER.info("Create topic {}, partitions {}, replicationFactor {}", topicName, partitions, replicationFactor); CompletableFuture promise = new CompletableFuture<>(); diff --git a/src/main/java/io/strimzi/kafka/bridge/http/HttpAdminBridgeEndpoint.java b/src/main/java/io/strimzi/kafka/bridge/http/HttpAdminBridgeEndpoint.java index 32208a9b..e4cc30e4 100644 --- a/src/main/java/io/strimzi/kafka/bridge/http/HttpAdminBridgeEndpoint.java +++ b/src/main/java/io/strimzi/kafka/bridge/http/HttpAdminBridgeEndpoint.java @@ -28,11 +28,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; @@ -184,8 +180,10 @@ public void doGetTopic(RoutingContext routingContext) { */ public void doCreateTopic(RoutingContext routingContext) { String topicName = routingContext.pathParam("topicname"); - int partitions = Integer.parseInt(routingContext.queryParams().get("partitions")); - short replicationFactor = Short.parseShort(routingContext.queryParams().get("replication_factor")); + Optional partitions = Optional.ofNullable(routingContext.queryParams().get("partitions")) + .map(Integer::valueOf); + Optional replicationFactor = Optional.ofNullable(routingContext.queryParams().get("replication_factor")) + .map(Short::valueOf); this.kafkaBridgeAdmin.createTopic(topicName, partitions, replicationFactor) .whenComplete(((topic, exception) -> { diff --git a/src/main/resources/openapi.json b/src/main/resources/openapi.json index 2f5749b2..59822465 100644 --- a/src/main/resources/openapi.json +++ b/src/main/resources/openapi.json @@ -615,7 +615,7 @@ } }, "404": { - "description": "The specified topic was not found.", + "description": "The path topic was not found.", "content": { "application/vnd.kafka.v2+json": { "schema": { @@ -772,7 +772,7 @@ } }, "404": { - "description": "The specified topic was not found.", + "description": "The path was not found.", "content": { "application/vnd.kafka.v2+json": { "schema": { @@ -782,7 +782,7 @@ "response": { "value": { "error_code": 404, - "message": "The specified topic was not found." + "message": "The path was not found." } } } @@ -795,7 +795,7 @@ { "name": "topicname", "in": "path", - "description": "Name of the topic to send records to or retrieve metadata from.", + "description": "Name of the topic will be created.", "required": true, "schema": { "type": "string" @@ -804,8 +804,8 @@ { "name": "partitions", "in": "query", - "description": "Number of partitions for the topic.", - "required": true, + "description": "Number of partitions for the topic. (Optional)", + "required": false, "schema": { "type": "integer" } @@ -813,8 +813,8 @@ { "name": "replication_factor", "in": "query", - "description": "Replication factor for the topic.", - "required": true, + "description": "Replication factor for the topic. (Optional)", + "required": false, "schema": { "type": "integer" } diff --git a/src/main/resources/openapiv2.json b/src/main/resources/openapiv2.json index 9d6f53c9..5009b40f 100644 --- a/src/main/resources/openapiv2.json +++ b/src/main/resources/openapiv2.json @@ -701,7 +701,7 @@ } }, "404": { - "description": "The specified topic was not found.", + "description": "The path was not found.", "schema": { "$ref": "#/definitions/Error" }, @@ -709,7 +709,7 @@ "response": { "value": { "error_code": 404, - "message": "The specified topic was not found." + "message": "The path was not found." } } } @@ -720,7 +720,7 @@ { "name": "topicname", "in": "path", - "description": "Name of the topic to send records to or retrieve metadata from.", + "description": "Name of the topic will be created.", "required": true, "schema": { "type": "string" @@ -729,8 +729,8 @@ { "name": "partitions", "in": "query", - "description": "Number of partitions for the topic.", - "required": true, + "description": "Number of partitions for the topic. (Optional)", + "required": false, "schema": { "type": "integer" } @@ -738,8 +738,8 @@ { "name": "replication_factor", "in": "query", - "description": "Replication factor for the topic.", - "required": true, + "description": "Replication factor for the topic. (Optional)", + "required": false, "schema": { "type": "integer" } diff --git a/src/test/java/io/strimzi/kafka/bridge/http/AdminClientIT.java b/src/test/java/io/strimzi/kafka/bridge/http/AdminClientIT.java index 43be8805..c8e5c642 100644 --- a/src/test/java/io/strimzi/kafka/bridge/http/AdminClientIT.java +++ b/src/test/java/io/strimzi/kafka/bridge/http/AdminClientIT.java @@ -218,6 +218,7 @@ void setupTopic(VertxTestContext context, String topic, int partitions, int coun @Test void createTopicTest(VertxTestContext context) { + // create topic test with 1 partition and 1 replication factor baseService() .postRequest("/admin/topics/" + topic) .addQueryParam("partitions", "1") @@ -231,5 +232,18 @@ void createTopicTest(VertxTestContext context) { }); context.completeNow(); }); + + // create topic test without partitions and replication factor + baseService() + .postRequest("/admin/topics/" + topic) + .as(BodyCodec.jsonArray()) + .send(ar -> { + context.verify(() -> { + assertThat(ar.succeeded(), is(true)); + HttpResponse response = ar.result(); + assertThat(response.statusCode(), is(HttpResponseStatus.OK.code())); + }); + context.completeNow(); + }); } }