diff --git a/docs/developer-guide/ksqldb-rest-api/info-endpoint.md b/docs/developer-guide/ksqldb-rest-api/info-endpoint.md index 60aa0b367032..f6a5af696b46 100644 --- a/docs/developer-guide/ksqldb-rest-api/info-endpoint.md +++ b/docs/developer-guide/ksqldb-rest-api/info-endpoint.md @@ -49,3 +49,26 @@ Your output should resemble: } ``` +To view non-sensitive server configurations, you can use the `/v1/configs` endpoint: + +```bash +curl -sX GET "http://localhost:8088/v1/configs" | jq '.' +``` + +Your output should resemble: + +```json +{ + "configs": { + "ksql.query.persistent.active.limit": 20 + } +} +``` + +To view a specific endpoint, you can add a query: + +```bash +curl -sX GET "http://localhost:8088/v1/configs?name=ksql.query.persistent.active.limit" | jq '.' +``` + +Currently, the only configuration that is visible is `ksql.query.persistent.active.limit`. \ No newline at end of file diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/ServerVerticle.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/ServerVerticle.java index 65839179ea1b..b6b5547b74c6 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/ServerVerticle.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/server/ServerVerticle.java @@ -41,6 +41,7 @@ import io.vertx.ext.web.RoutingContext; import io.vertx.ext.web.handler.BodyHandler; import java.nio.channels.ClosedChannelException; +import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -205,7 +206,10 @@ private Router setupRouter() { .produces(Versions.KSQL_V1_JSON) .produces(JSON_CONTENT_TYPE) .handler(this::handleWebsocket); - + router.route(HttpMethod.GET, "/v1/configs") + .produces(Versions.KSQL_V1_JSON) + .produces(JSON_CONTENT_TYPE) + .handler(this::handleConfigRequest); return router; } @@ -326,6 +330,15 @@ private void handleWebsocket(final RoutingContext routingContext) { server.getWorkerExecutor(), apiSecurityContext); } + private void handleConfigRequest(final RoutingContext routingContext) { + final List requestedConfigs = routingContext.queryParam("name"); + handleOldApiRequest(server, routingContext, null, Optional.empty(), + (request, apiSecurityContext) -> + endpoints + .executeConfig(requestedConfigs, DefaultApiSecurityContext.create(routingContext)) + ); + } + private static void chcHandler(final RoutingContext routingContext) { routingContext.response().putHeader(HttpHeaders.CONTENT_TYPE.toString(), "application/json") .end(new JsonObject().toBuffer()); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/spi/Endpoints.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/spi/Endpoints.java index f881735705e4..160827145cfd 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/spi/Endpoints.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/spi/Endpoints.java @@ -28,6 +28,7 @@ import io.vertx.core.WorkerExecutor; import io.vertx.core.http.ServerWebSocket; import io.vertx.core.json.JsonObject; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.reactivestreams.Subscriber; @@ -105,6 +106,9 @@ CompletableFuture executeLagReport(LagReportingMessage lagRepo CompletableFuture executeServerMetadataClusterId( ApiSecurityContext apiSecurityContext); + CompletableFuture executeConfig( + List requestedConfigs, ApiSecurityContext apiSecurityContext); + // This is the legacy websocket based query streaming API void executeWebsocketStream(ServerWebSocket webSocket, MultiMap requstParams, WorkerExecutor workerExecutor, ApiSecurityContext apiSecurityContext); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index 335065b17ba3..d8071f20f31a 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -69,6 +69,7 @@ import io.confluent.ksql.rest.server.execution.PullQueryExecutor; import io.confluent.ksql.rest.server.execution.PullQueryExecutorMetrics; import io.confluent.ksql.rest.server.resources.ClusterStatusResource; +import io.confluent.ksql.rest.server.resources.ConfigResource; import io.confluent.ksql.rest.server.resources.HealthCheckResource; import io.confluent.ksql.rest.server.resources.HeartbeatResource; import io.confluent.ksql.rest.server.resources.KsqlConfigurable; @@ -183,6 +184,7 @@ public final class KsqlRestApplication implements Executable { private final HealthCheckResource healthCheckResource; private volatile ServerMetadataResource serverMetadataResource; private volatile WSQueryEndpoint wsQueryEndpoint; + private final ConfigResource configResource; @SuppressWarnings("UnstableApiUsage") private volatile ListeningScheduledExecutorService oldApiWebsocketExecutor; private final Vertx vertx; @@ -280,6 +282,7 @@ public static SourceName getCommandsStreamName() { this.ksqlConfigNoPort, this.commandRunner); this.queryMonitor = requireNonNull(ksqlQueryMonitor, "ksqlQueryMonitor"); + this.configResource = new ConfigResource(ksqlConfig); MetricCollectors.addConfigurableReporter(ksqlConfigNoPort); this.pullQueryMetrics = requireNonNull(pullQueryMetrics, "pullQueryMetrics"); log.debug("ksqlDB API server instance created"); @@ -342,6 +345,7 @@ public void startAsync() { healthCheckResource, serverMetadataResource, wsQueryEndpoint, + configResource, pullQueryMetrics ); apiServer = new Server(vertx, ksqlRestConfig, endpoints, securityExtension, diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerEndpoints.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerEndpoints.java index a7067aa17d01..b30ca93db996 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerEndpoints.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerEndpoints.java @@ -34,6 +34,7 @@ import io.confluent.ksql.rest.server.execution.PullQueryExecutor; import io.confluent.ksql.rest.server.execution.PullQueryExecutorMetrics; import io.confluent.ksql.rest.server.resources.ClusterStatusResource; +import io.confluent.ksql.rest.server.resources.ConfigResource; import io.confluent.ksql.rest.server.resources.HealthCheckResource; import io.confluent.ksql.rest.server.resources.HeartbeatResource; import io.confluent.ksql.rest.server.resources.KsqlResource; @@ -52,6 +53,7 @@ import io.vertx.core.WorkerExecutor; import io.vertx.core.http.ServerWebSocket; import io.vertx.core.json.JsonObject; +import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -78,6 +80,7 @@ public class KsqlServerEndpoints implements Endpoints { private final HealthCheckResource healthCheckResource; private final ServerMetadataResource serverMetadataResource; private final WSQueryEndpoint wsQueryEndpoint; + private final ConfigResource configResource; private final Optional pullQueryMetrics; // CHECKSTYLE_RULES.OFF: ParameterNumber @@ -96,6 +99,7 @@ public KsqlServerEndpoints( final HealthCheckResource healthCheckResource, final ServerMetadataResource serverMetadataResource, final WSQueryEndpoint wsQueryEndpoint, + final ConfigResource configResource, final Optional pullQueryMetrics ) { @@ -115,6 +119,7 @@ public KsqlServerEndpoints( this.healthCheckResource = Objects.requireNonNull(healthCheckResource); this.serverMetadataResource = Objects.requireNonNull(serverMetadataResource); this.wsQueryEndpoint = Objects.requireNonNull(wsQueryEndpoint); + this.configResource = Objects.requireNonNull(configResource); this.pullQueryMetrics = Objects.requireNonNull(pullQueryMetrics); } @@ -253,6 +258,19 @@ public CompletableFuture executeServerMetadataClusterId( ksqlSecurityContext -> serverMetadataResource.getServerClusterId()); } + @Override + public CompletableFuture executeConfig( + final List requestedConfigs, + final ApiSecurityContext apiSecurityContext) { + if (requestedConfigs.size() == 0) { + return executeOldApiEndpoint(apiSecurityContext, + ksqlSecurityContext -> configResource.getAllConfigs()); + } else { + return executeOldApiEndpoint(apiSecurityContext, + ksqlSecurityContext -> configResource.getConfigs(requestedConfigs)); + } + } + @Override public void executeWebsocketStream(final ServerWebSocket webSocket, final MultiMap requestParams, final WorkerExecutor workerExecutor, diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ConfigResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ConfigResource.java new file mode 100644 index 000000000000..7ff3ef897f58 --- /dev/null +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/ConfigResource.java @@ -0,0 +1,53 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.resources; + +import io.confluent.ksql.rest.EndpointResponse; +import io.confluent.ksql.rest.entity.ConfigResponse; +import io.confluent.ksql.util.KsqlConfig; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ConfigResource { + private final Map visibleConfigs = new HashMap<>(); + + public ConfigResource(final KsqlConfig ksqlConfig) { + setVisibleConfigs(ksqlConfig); + } + + private void setVisibleConfigs(final KsqlConfig ksqlConfig) { + visibleConfigs.put( + KsqlConfig.KSQL_ACTIVE_PERSISTENT_QUERY_LIMIT_CONFIG, + ksqlConfig.getInt(KsqlConfig.KSQL_ACTIVE_PERSISTENT_QUERY_LIMIT_CONFIG) + ); + } + + public EndpointResponse getConfigs(final List requestedConfigs) { + final Map configs = new HashMap<>(); + for (String config : requestedConfigs) { + final Object value = visibleConfigs.get(config); + if (value != null) { + configs.put(config, value); + } + } + return EndpointResponse.ok(new ConfigResponse(configs)); + } + + public EndpointResponse getAllConfigs() { + return EndpointResponse.ok(new ConfigResponse(visibleConfigs)); + } +} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java index 8dc576ae9a6d..6edaffb53ced 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/TestEndpoints.java @@ -195,6 +195,12 @@ public CompletableFuture executeServerMetadataClusterId( return null; } + @Override + public CompletableFuture executeConfig( + List requestedConfigs, ApiSecurityContext apiSecurityContext) { + return null; + } + @Override public void executeWebsocketStream(ServerWebSocket webSocket, MultiMap requstParams, WorkerExecutor workerExecutor, ApiSecurityContext apiSecurityContext) { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java index b274e36cb6a2..a87d61d3068a 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/InsertsStreamRunner.java @@ -39,6 +39,7 @@ import io.vertx.core.parsetools.RecordParser; import io.vertx.core.streams.ReadStream; import io.vertx.ext.web.codec.BodyCodec; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.reactivestreams.Subscriber; @@ -247,6 +248,13 @@ public CompletableFuture executeServerMetadataClusterId( return null; } + @Override + public CompletableFuture executeConfig( + List requestedConfigs, + ApiSecurityContext apiSecurityContext) { + return null; + } + @Override public void executeWebsocketStream(ServerWebSocket webSocket, MultiMap requstParams, WorkerExecutor workerExecutor, ApiSecurityContext apiSecurityContext) { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java index 7cc2d26d40ca..4d0be3cd5b14 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/PullQueryRunner.java @@ -204,6 +204,13 @@ public CompletableFuture executeServerMetadataClusterId( return null; } + @Override + public CompletableFuture executeConfig( + List requestedConfigs, + ApiSecurityContext apiSecurityContext) { + return null; + } + @Override public void executeWebsocketStream(ServerWebSocket webSocket, MultiMap requstParams, WorkerExecutor workerExecutor, ApiSecurityContext apiSecurityContext) { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/QueryStreamRunner.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/QueryStreamRunner.java index eb1a7345036e..1096b592a17c 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/QueryStreamRunner.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/perf/QueryStreamRunner.java @@ -192,6 +192,13 @@ public CompletableFuture executeServerMetadataClusterId( return null; } + @Override + public CompletableFuture executeConfig( + List requestedConfigs, + ApiSecurityContext apiSecurityContext) { + return null; + } + @Override public void executeWebsocketStream(ServerWebSocket webSocket, MultiMap requstParams, WorkerExecutor workerExecutor, ApiSecurityContext apiSecurityContext) { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java index 67ab67671552..c7f156878c00 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java @@ -50,6 +50,7 @@ import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatus.Status; import io.confluent.ksql.rest.entity.CommandStatuses; +import io.confluent.ksql.rest.entity.ConfigResponse; import io.confluent.ksql.rest.entity.KsqlRequest; import io.confluent.ksql.rest.entity.ServerClusterId; import io.confluent.ksql.rest.entity.ServerInfo; @@ -62,6 +63,7 @@ import io.confluent.ksql.test.util.secure.ClientTrustStore; import io.confluent.ksql.test.util.secure.Credentials; import io.confluent.ksql.test.util.secure.SecureKafkaHelper; +import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.PageViewDataProvider; import io.vertx.core.buffer.Buffer; import io.vertx.core.http.HttpMethod; @@ -289,6 +291,28 @@ public void shouldExecuteServerMetadataIdRequest() { assertThat(response, is(notNullValue())); } + @Test + public void shouldExecuteAllConfigsRequest() { + // When: + final ConfigResponse response = RestIntegrationTestUtil.makeConfigRequest(REST_APP); + + // Then: + assertThat(response.getConfigs().get(KsqlConfig.KSQL_ACTIVE_PERSISTENT_QUERY_LIMIT_CONFIG), is(notNullValue())); + } + + @Test + public void shouldExecuteConfigRequest() { + // When: + final ConfigResponse response = RestIntegrationTestUtil.makeConfigRequest( + REST_APP, + Arrays.asList(new String[] {"foo", KsqlConfig.CONNECT_URL_PROPERTY, KsqlConfig.KSQL_ACTIVE_PERSISTENT_QUERY_LIMIT_CONFIG}) + ); + + // Then: + assertThat(response.getConfigs().keySet().size(), is(1)); + assertThat(response.getConfigs().get(KsqlConfig.KSQL_ACTIVE_PERSISTENT_QUERY_LIMIT_CONFIG), is(notNullValue())); + } + @Test public void shouldExecuteRootDocumentRequest() { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java index f84429e9fa70..a138d2efa894 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java @@ -29,6 +29,7 @@ import io.confluent.ksql.rest.entity.CommandStatus.Status; import io.confluent.ksql.rest.entity.CommandStatusEntity; import io.confluent.ksql.rest.entity.CommandStatuses; +import io.confluent.ksql.rest.entity.ConfigResponse; import io.confluent.ksql.rest.entity.KsqlEntity; import io.confluent.ksql.rest.entity.KsqlEntityList; import io.confluent.ksql.rest.entity.KsqlErrorMessage; @@ -155,6 +156,31 @@ static ServerClusterId makeServerMetadataIdRequest(final TestKsqlRestApp restApp } } + static ConfigResponse makeConfigRequest(final TestKsqlRestApp restApp) { + try (final KsqlRestClient restClient = restApp.buildKsqlClient(Optional.empty())) { + + final RestResponse res = restClient.makeConfigRequest(); + + throwOnError(res); + + return res.getResponse(); + } + } + + static ConfigResponse makeConfigRequest( + final TestKsqlRestApp restApp, + final List requestedConfigs + ) { + try (final KsqlRestClient restClient = restApp.buildKsqlClient(Optional.empty())) { + + final RestResponse res = restClient.makeConfigRequest(requestedConfigs); + + throwOnError(res); + + return res.getResponse(); + } + } + static List makeQueryRequest( final TestKsqlRestApp restApp, final String sql, diff --git a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java index 0c91a7613372..77a0aaad5c9d 100644 --- a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java +++ b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlRestClient.java @@ -23,6 +23,7 @@ import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatuses; +import io.confluent.ksql.rest.entity.ConfigResponse; import io.confluent.ksql.rest.entity.HealthCheckResponse; import io.confluent.ksql.rest.entity.HeartbeatResponse; import io.confluent.ksql.rest.entity.KsqlEntityList; @@ -149,6 +150,14 @@ public RestResponse makeClusterStatusRequest() { return target().getClusterStatus(); } + public RestResponse makeConfigRequest() { + return target().getConfigRequest(); + } + + public RestResponse makeConfigRequest(final List requestedConfigs) { + return target().getConfigRequest(requestedConfigs); + } + public CompletableFuture> makeAsyncLagReportingRequest( final LagReportingMessage lagReportingMessage ) { diff --git a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java index 5589139a2b93..aa2ffb8f4979 100644 --- a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java +++ b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTarget.java @@ -23,6 +23,7 @@ import io.confluent.ksql.rest.entity.ClusterStatusResponse; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatuses; +import io.confluent.ksql.rest.entity.ConfigResponse; import io.confluent.ksql.rest.entity.HealthCheckResponse; import io.confluent.ksql.rest.entity.HeartbeatMessage; import io.confluent.ksql.rest.entity.HeartbeatResponse; @@ -52,6 +53,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Function; +import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,6 +70,7 @@ public final class KsqlTarget { private static final String LAG_REPORT_PATH = "/lag"; private static final String SERVER_METADATA_PATH = "/v1/metadata"; private static final String SERVER_METADATA_ID_PATH = "/v1/metadata/id"; + private static final String CONFIG_PATH = "/v1/configs"; private final HttpClient httpClient; private final SocketAddress socketAddress; @@ -132,6 +135,18 @@ public RestResponse getClusterStatus() { return get(CLUSTERSTATUS_PATH, ClusterStatusResponse.class); } + public RestResponse getConfigRequest() { + return get(CONFIG_PATH, ConfigResponse.class); + } + + public RestResponse getConfigRequest(final List requestedConfigs) { + return get( + CONFIG_PATH + requestedConfigs.stream() + .collect(Collectors.joining("&name=", "?name=", "")), + ConfigResponse.class + ); + } + public CompletableFuture> postAsyncLagReportingRequest( final LagReportingMessage lagReportingMessage ) { diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/ConfigResponse.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/ConfigResponse.java new file mode 100644 index 000000000000..8ca40b61669c --- /dev/null +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/ConfigResponse.java @@ -0,0 +1,64 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.entity; + +import static java.util.Objects.requireNonNull; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import java.util.Map; +import java.util.Objects; + +@JsonIgnoreProperties(ignoreUnknown = true) +@JsonSubTypes({}) +public class ConfigResponse { + private final Map configs; + + @JsonCreator + public ConfigResponse(@JsonProperty("configs") final Map configs) { + this.configs = requireNonNull(configs, "configs"); + } + + public Map getConfigs() { + return configs; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final ConfigResponse that = (ConfigResponse) o; + return Objects.equals(configs, that.configs); + } + + @Override + public int hashCode() { + return Objects.hash(configs); + } + + @Override + public String toString() { + return "configs = " + configs; + } +}